From 21dce790ae7b1e172082c290c5a9aa48bba2843d Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Sun, 2 Nov 2025 16:02:31 +0900 Subject: [PATCH 001/400] [SPARK-54052][PYTHON] Add a bridge object to workaround Py4J limitation ### What changes were proposed in this pull request? This PR proposes to add PythonErrorUtils object to workaround Py4J limitation. Py4J does not support default method access. ### Why are the changes needed? To make the change easier and non error prone ### Does this PR introduce _any_ user-facing change? No. Virtually a refactoring change. ### How was this patch tested? Unittest was added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52755 from HyukjinKwon/bridge-class. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../spark/api/python/PythonErrorUtils.scala | 41 ++++++++++++++++ .../spark/deploy/PythonRunnerSuite.scala | 30 +++++++++++- python/pyspark/errors/exceptions/captured.py | 49 +++++-------------- 3 files changed, 82 insertions(+), 38 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/api/python/PythonErrorUtils.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonErrorUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonErrorUtils.scala new file mode 100644 index 0000000000000..73c2a29ea4095 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonErrorUtils.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import java.util + +import org.apache.spark.{BreakingChangeInfo, QueryContext, SparkThrowable} + +/** + * Utility object that provides convenient accessors for extracting + * detailed information from a [[SparkThrowable]] instance. + * + * This object is primarily used in PySpark + * to retrieve structured error metadata because Py4J does not work + * with default methods. + */ +private[spark] object PythonErrorUtils { + def getCondition(e: SparkThrowable): String = e.getCondition + def getErrorClass(e: SparkThrowable): String = e.getCondition + def getSqlState(e: SparkThrowable): String = e.getSqlState + def isInternalError(e: SparkThrowable): Boolean = e.isInternalError + def getBreakingChangeInfo(e: SparkThrowable): BreakingChangeInfo = e.getBreakingChangeInfo + def getMessageParameters(e: SparkThrowable): util.Map[String, String] = e.getMessageParameters + def getDefaultMessageTemplate(e: SparkThrowable): String = e.getDefaultMessageTemplate + def getQueryContext(e: SparkThrowable): Array[QueryContext] = e.getQueryContext +} diff --git a/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala index 473a2d7b2a258..2cce3d306e60c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.deploy -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkThrowable} +import org.apache.spark.api.python.PythonErrorUtils import org.apache.spark.util.Utils class PythonRunnerSuite extends SparkFunSuite { @@ -64,4 +65,31 @@ class PythonRunnerSuite extends SparkFunSuite { intercept[IllegalArgumentException] { PythonRunner.formatPaths("hdfs:/some.py,foo.py") } intercept[IllegalArgumentException] { PythonRunner.formatPaths("foo.py,hdfs:/some.py") } } + + test("SPARK-54052: PythonErrorUtils should have corresponding methods in SparkThrowable") { + // Find default methods in SparkThrowable + val defaultMethods = classOf[SparkThrowable] + .getMethods + .filter(m => m.getDeclaringClass == classOf[SparkThrowable]) + .map(_.getName) + .toSet + + // Find methods defined in PythonErrorUtils object + val utilsMethods = PythonErrorUtils.getClass + .getDeclaredMethods + .filterNot(_.isSynthetic) + .map(_.getName) + .filterNot(_.contains("$")) + .toSet + + // Compare + assert( + utilsMethods == defaultMethods, + s""" + |PythonErrorUtils methods and SparkThrowable default methods differ! + |Missing in PythonErrorUtils: ${defaultMethods.diff(utilsMethods).mkString(", ")} + |Extra in PythonErrorUtils: ${utilsMethods.diff(defaultMethods).mkString(", ")} + |""".stripMargin + ) + } } diff --git a/python/pyspark/errors/exceptions/captured.py b/python/pyspark/errors/exceptions/captured.py index 56892db91f3be..0f76e3b5f6a07 100644 --- a/python/pyspark/errors/exceptions/captured.py +++ b/python/pyspark/errors/exceptions/captured.py @@ -107,7 +107,8 @@ def getCondition(self) -> Optional[str]: if self._origin is not None and is_instance_of( gw, self._origin, "org.apache.spark.SparkThrowable" ): - return self._origin.getCondition() + utils = SparkContext._jvm.PythonErrorUtils # type: ignore[union-attr] + return utils.getCondition(self._origin) else: return None @@ -118,7 +119,6 @@ def getErrorClass(self) -> Optional[str]: def getMessageParameters(self) -> Optional[Dict[str, str]]: from pyspark import SparkContext from py4j.java_gateway import is_instance_of - from py4j.protocol import Py4JError assert SparkContext._gateway is not None @@ -126,38 +126,28 @@ def getMessageParameters(self) -> Optional[Dict[str, str]]: if self._origin is not None and is_instance_of( gw, self._origin, "org.apache.spark.SparkThrowable" ): - try: - return dict(self._origin.getMessageParameters()) - except Py4JError as e: - if "py4j.Py4JException" in str(e) and "Method getMessageParameters" in str(e): - return None - raise e + utils = SparkContext._jvm.PythonErrorUtils # type: ignore[union-attr] + return dict(utils.getMessageParameters(self._origin)) else: return None def getSqlState(self) -> Optional[str]: from pyspark import SparkContext from py4j.java_gateway import is_instance_of - from py4j.protocol import Py4JError assert SparkContext._gateway is not None gw = SparkContext._gateway if self._origin is not None and is_instance_of( gw, self._origin, "org.apache.spark.SparkThrowable" ): - try: - return self._origin.getSqlState() - except Py4JError as e: - if "py4j.Py4JException" in str(e) and "Method getSqlState" in str(e): - return None - raise e + utils = SparkContext._jvm.PythonErrorUtils # type: ignore[union-attr] + return utils.getSqlState(self._origin) else: return None def getMessage(self) -> str: from pyspark import SparkContext from py4j.java_gateway import is_instance_of - from py4j.protocol import Py4JError assert SparkContext._gateway is not None gw = SparkContext._gateway @@ -165,21 +155,12 @@ def getMessage(self) -> str: if self._origin is not None and is_instance_of( gw, self._origin, "org.apache.spark.SparkThrowable" ): - try: - error_class = self._origin.getCondition() - except Py4JError as e: - if "py4j.Py4JException" in str(e) and "Method getCondition" in str(e): - return "" - raise e - try: - message_parameters = self._origin.getMessageParameters() - except Py4JError as e: - if "py4j.Py4JException" in str(e) and "Method getMessageParameters" in str(e): - return "" - raise e + utils = SparkContext._jvm.PythonErrorUtils # type: ignore[union-attr] + errorClass = utils.getCondition(self._origin) + messageParameters = utils.getMessageParameters(self._origin) error_message = getattr(gw.jvm, "org.apache.spark.SparkThrowableHelper").getMessage( - error_class, message_parameters + errorClass, messageParameters ) return error_message @@ -189,7 +170,6 @@ def getMessage(self) -> str: def getQueryContext(self) -> List[BaseQueryContext]: from pyspark import SparkContext from py4j.java_gateway import is_instance_of - from py4j.protocol import Py4JError assert SparkContext._gateway is not None @@ -198,13 +178,8 @@ def getQueryContext(self) -> List[BaseQueryContext]: gw, self._origin, "org.apache.spark.SparkThrowable" ): contexts: List[BaseQueryContext] = [] - try: - context = self._origin.getQueryContext() - except Py4JError as e: - if "py4j.Py4JException" in str(e) and "Method getQueryContext" in str(e): - return [] - raise e - for q in context: + utils = SparkContext._jvm.PythonErrorUtils # type: ignore[union-attr] + for q in utils.getQueryContext(self._origin): if q.contextType().toString() == "SQL": contexts.append(SQLQueryContext(q)) else: From 55b7c6340d33266a4dd9397202889d8667141293 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Sun, 2 Nov 2025 16:49:41 +0900 Subject: [PATCH 002/400] [SPARK-54123][PYTHON] Add timezone to make the timestamp an absolute time ### What changes were proposed in this pull request? Adds timezone to make the timestamp in the log record JSON string the absolute time. ### Why are the changes needed? Without the timezone, the timestamp of each log record doesn't reflect the session timezone, which makes it confusing.
example ```python >>> from pyspark.sql.functions import * >>> import logging >>> >>> udf ... def logging_test_udf(x): ... logger = logging.getLogger("test") ... logger.warning(f"message") ... return str(x) ... >>> >>> spark.conf.set("spark.sql.pyspark.worker.logging.enabled", True) >>> >>> spark.range(1).select(logging_test_udf("id")).show() ... ```
- Before ```python >>> spark.conf.get('spark.sql.session.timeZone') 'America/Los_Angeles' >>> spark.sql("select ts from system.session.python_worker_logs").show(truncate=False) +--------------------------+ |ts | +--------------------------+ |2025-10-31 17:17:59.495541| +--------------------------+ >>> spark.conf.set('spark.sql.session.timeZone', 'UTC') >>> spark.sql("select ts from system.session.python_worker_logs").show(truncate=False) +--------------------------+ |ts | +--------------------------+ |2025-10-31 17:17:59.495541| +--------------------------+ ``` - After ```python >>> spark.conf.get('spark.sql.session.timeZone') 'America/Los_Angeles' >>> spark.sql("select ts from system.session.python_worker_logs").show(truncate=False) +--------------------------+ |ts | +--------------------------+ |2025-10-31 17:19:52.152868| +--------------------------+ >>> spark.conf.set('spark.sql.session.timeZone', 'UTC') >>> spark.sql("select ts from system.session.python_worker_logs").show(truncate=False) +--------------------------+ |ts | +--------------------------+ |2025-11-01 00:19:52.152868| +--------------------------+ ``` ### Does this PR introduce _any_ user-facing change? Yes, the timestamp of log record is now absolute time. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52823 from ueshin/issues/SPARK-54123/timezone. Authored-by: Takuya Ueshin Signed-off-by: Hyukjin Kwon (cherry picked from commit e837ff9707ef5d08cde53c338dfaf7d81a5c55aa) Signed-off-by: Hyukjin Kwon --- python/pyspark/logger/worker_io.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/logger/worker_io.py b/python/pyspark/logger/worker_io.py index 2e5ced2e84ad3..79684b7aca624 100644 --- a/python/pyspark/logger/worker_io.py +++ b/python/pyspark/logger/worker_io.py @@ -164,6 +164,7 @@ def formatTime(self, record: logging.LogRecord, datefmt: Optional[str] = None) - ) elif self.default_msec_format: s = self.default_msec_format % (s, record.msecs) + s = f"{s}{time.strftime('%z', ct)}" return s From 5f3fe121570bb8c74f5d64d7a4fea5305bb7221b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 2 Nov 2025 08:29:10 -0800 Subject: [PATCH 003/400] [SPARK-54135][INFRA][4.1] Add GitHub Action jobs for branch-4.1 ### What changes were proposed in this pull request? This PR aims to add `GitHub Action` jobs to branch-4.1. ### Why are the changes needed? To trigger CIs on branch-4.1 PR and commits. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52834 from dongjoon-hyun/SPARK-54135. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 3776a116fd785..3ba71108f5533 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -30,8 +30,7 @@ on: description: Branch to run the build against required: false type: string - # Change 'master' to 'branch-4.0' in branch-4.0 branch after cutting it. - default: master + default: branch-4.1 hadoop: description: Hadoop version to run with. HADOOP_PROFILE environment variable should accept it. required: false From a5d8441237887fec9e226db17b3d95783fe462b2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 2 Nov 2025 13:42:09 -0800 Subject: [PATCH 004/400] [SPARK-54140][INFRA] Add `libwebp-dev` to fix `dev/spark-test-image/lint/Dockerfile` building ### What changes were proposed in this pull request? This PR aims to add `libwebp-dev` to fix `dev/spark-test-image/lint/Dockerfile` building in both `master` and `branch-4.1`. ### Why are the changes needed? Currently, `dev/spark-test-image/lint/Dockerfile` fails to build. - For master branch, it wasn't revealed yet because we use the cached image. - For `branch-4.1`, it is currently breaking the CIs. - https://github.com/apache/spark/tree/branch-4.1 - https://github.com/apache/spark/actions/runs/19015025991/job/54307102990 ``` #9 454.6 -------------------------- [ERROR MESSAGE] --------------------------- #9 454.6 :1:10: fatal error: ft2build.h: No such file or directory #9 454.6 compilation terminated. #9 454.6 -------------------------------------------------------------------- #9 454.6 ERROR: configuration failed for package 'ragg' #9 454.6 * removing '/usr/local/lib/R/site-library/ragg' ``` ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs. Especially, `Base image build` job. - https://github.com/dongjoon-hyun/spark/actions/runs/19018354185/job/54309542386 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52838 from dongjoon-hyun/SPARK-54140. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 25aa49a3cdfb84c2038289aab3b4f7b4a8445d2c) Signed-off-by: Dongjoon Hyun --- dev/spark-test-image/lint/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 3b603d4ab4a68..07ff9c90b7591 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -46,6 +46,7 @@ RUN apt-get update && apt-get install -y \ libpng-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ nodejs \ npm \ From 421c474832da2da5411ab07c611b02f4f35c9cf5 Mon Sep 17 00:00:00 2001 From: Xiaonan Yang Date: Mon, 3 Nov 2025 08:03:08 +0900 Subject: [PATCH 005/400] [SPARK-54099][SQL] XML variant parser should fall back to string on decimal parsing errors ### What changes were proposed in this pull request? When parsing XML data with `parse_xml` that contains decimal numbers with very large exponents (e.g., "1E+2147483647"), the conversion to Variant type fails with: ``` java.lang.ArithmeticException: BigInteger would overflow supported range at java.base/java.math.BigDecimal.setScale(BigDecimal.java:3000) at org.apache.spark.sql.catalyst.xml.StaxXmlParser$.org$apache$spark$sql$catalyst$xml$StaxXmlParser$$appendXMLCharacterToVariant(StaxXmlParser.scala:1335) ``` It's because the parser calls `setScale(0)` to normalize the decimal. When the scale is extremely negative (e.g., -2147483647), `setScale(0)` attempts to multiply the unscaled value by 10^2147483647, causing BigInteger overflow. This PR will catch all errors when parsing strings as decimal in the XML variant parser and fall back to string. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52801 from xiaonanyang-db/SPARK-54099. Authored-by: Xiaonan Yang Signed-off-by: Hyukjin Kwon (cherry picked from commit 88eef06c221d9e1ec9e64e17296c20733572e3aa) Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/xml/StaxXmlParser.scala | 25 ++++++++++--------- .../datasources/xml/XmlVariantSuite.scala | 20 +++++++++++++++ 2 files changed, 33 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala index 1a47fa7bd43f0..3d6c57c9f7465 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala @@ -1278,18 +1278,19 @@ object StaxXmlParser { // Try parsing the value as decimal val decimalParser = ExprUtils.getDecimalParser(options.locale) - allCatch opt decimalParser(value) match { - case Some(decimalValue) => - var d = decimalValue - if (d.scale() < 0) { - d = d.setScale(0) - } - if (d.scale <= VariantUtil.MAX_DECIMAL16_PRECISION && - d.precision <= VariantUtil.MAX_DECIMAL16_PRECISION) { - builder.appendDecimal(d) - return - } - case _ => + try { + var d = decimalParser(value) + if (d.scale() < 0) { + d = d.setScale(0) + } + if (d.scale <= VariantUtil.MAX_DECIMAL16_PRECISION && + d.precision <= VariantUtil.MAX_DECIMAL16_PRECISION) { + builder.appendDecimal(d) + return + } + } catch { + case NonFatal(_) => + // Ignore the exception and parse it as a string below } // If the character is of other primitive types, parse it as a string diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala index 5f5e017038b74..15713e759b81b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala @@ -942,6 +942,26 @@ class XmlVariantSuite extends QueryTest with SharedSparkSession with TestXmlData .map(_.getString(0).replaceAll("\\s+", "")) assert(xmlResult.head === xmlStr) } + + test( + "[SPARK-54099] XML variant parser should fall back to string " + + "when failing to parse decimal values" + ) { + // Decimals with extreme exponents. The variant parser should throw ArithmeticException when + // parsing these values as Decimal: + val decimalString = Seq( + "1E+2147483647", // Maximum int exponent - scale would be -2147483647 + "5E+1000000000", // 1 billion exponent + "1.23E+999999999", // Very large exponent + "0.001E+2147483640" // Still results in huge effective exponent + ) + decimalString.foreach { str => + testParser( + xml = s"$str", + expectedJsonStr = s"""{"decimal":"$str"}""" + ) + } + } } class XmlVariantSuiteWithLegacyParser extends XmlVariantSuite { From bcb5b996dc8e14f82a9491100255c8d96415ea42 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 2 Nov 2025 15:05:06 -0800 Subject: [PATCH 006/400] [SPARK-54141][INFRA] Add `libwebp-dev` to fix `dev/infra/Dockerfile` building ### What changes were proposed in this pull request? This PR aims to add `libwebp-dev` to fix `dev/infra/Dockerfile` building. ### Why are the changes needed? To fix `build_infra_images_cache` GitHub Action job - https://github.com/apache/spark/actions/workflows/build_infra_images_cache.yml Screenshot 2025-11-02 at 14 56 19 The root cause is identical with other Dockerfile failure. ``` #13 578.4 -------------------------- [ERROR MESSAGE] --------------------------- #13 578.4 :1:10: fatal error: ft2build.h: No such file or directory #13 578.4 compilation terminated. #13 578.4 -------------------------------------------------------------------- #13 578.4 ERROR: configuration failed for package 'ragg' ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. Especially, `Cache base image` test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52840 from dongjoon-hyun/SPARK-54141. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 689851eae71a04cfe03baebbd8b6beb526f94f23) Signed-off-by: Dongjoon Hyun --- dev/infra/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 873d572063118..afab64a67b5e5 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ nodejs \ npm \ From b5bc199284406e4c9f300993ff51f1d9cc23b7d3 Mon Sep 17 00:00:00 2001 From: "cafri.sun" Date: Sun, 2 Nov 2025 15:18:30 -0800 Subject: [PATCH 007/400] [SPARK-39328][SQL][TESTS] Fix flaky test `SPARK-37753: Inhibit broadcast in left outer join when there are many empty partitions on outer/left side` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Improve test `SPARK-37753: Inhibit broadcast in left outer join when there are many empty partitions on outer/left side` of `AdaptiveQueryExecSuite` ### Why are the changes needed? This test appears to always succeed in the Apache GitHub Action runner environment, But some environments, test does not seem to proceed as intended. On my environment: `4.18.0-553.8.1.el8_10.x86_64` `Intel(R) Xeon(R) Silver 4210 CPU 2.20GHz` `64G Mem` And ran test in master branch following the guide of official documentation ``` ./build/sbt testOnly org.apache.spark.sql.execution.adaptive.AdaptiveQueryExecSuite ... - SPARK-37753: Inhibit broadcast in left outer join when there are many empty partitions on outer/left side *** FAILED *** The code passed to eventually never returned normally. Attempted 25 times over 15.040156205999999 seconds. Last failure message: ``` even increasing the test's timeout to 1500 seconds results to failure after lots of retries. ``` SPARK-37753: Inhibit broadcast in left outer join when there are many empty partitions on outer/left side *** FAILED *** The code passed to failAfter did not complete within 20 minutes. (AdaptiveQueryExecSuite.scala:743) ``` --- The test says ```scala // if the right side is completed first and the left side is still being executed, // the right side does not know whether there are many empty partitions on the left side, // so there is no demote, and then the right side is broadcast in the planning stage. // so retry several times here to avoid unit test failure. eventually(timeout(15.seconds), interval(500.milliseconds)) { ... ``` It seems test failure occurs with very high probability by loading the ‘right side’ completes first. While the reason is unclear, I believe it would be better to regulate the subquery loading speed in a predictable manner via applying simple udf rather than retrying until both sides load in the desired order. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Rerun the test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52388 from Last-remote11/SPARK-39328. Authored-by: cafri.sun Signed-off-by: Dongjoon Hyun (cherry picked from commit e7b7acfd44f0ee9df424d1629060e776fadc1c99) Signed-off-by: Dongjoon Hyun --- .../execution/adaptive/AdaptiveQueryExecSuite.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index cede78c0ef3cd..3e7d26f74bd46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -22,7 +22,6 @@ import java.net.URI import org.apache.logging.log4j.Level import org.scalatest.PrivateMethodTester -import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -747,8 +746,13 @@ class AdaptiveQueryExecSuite // if the right side is completed first and the left side is still being executed, // the right side does not know whether there are many empty partitions on the left side, // so there is no demote, and then the right side is broadcast in the planning stage. - // so retry several times here to avoid unit test failure. - eventually(timeout(15.seconds), interval(500.milliseconds)) { + // so apply `slow_udf` to delay right side to avoid unit test failure. + withUserDefinedFunction("slow_udf" -> true) { + spark.udf.register("slow_udf", (x: Int) => { + Thread.sleep(300) + x + }) + withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5") { @@ -756,7 +760,8 @@ class AdaptiveQueryExecSuite withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "200") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM (select * from testData where value = '1') td" + - " left outer join testData2 ON key = a") + " left outer join (select slow_udf(a) as a, b from testData2) as td2" + + " ON td.key = td2.a") val smj = findTopLevelSortMergeJoin(plan) assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) From d160af39fea29c020e414ca18cb9a8340115110f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Sun, 2 Nov 2025 20:24:19 -0800 Subject: [PATCH 008/400] [MINOR][PYTHON][DOCS] Fix the doctest of `pivot` ### What changes were proposed in this pull request? ### Why are the changes needed? Fix the doctest of `pivot`, to make sure the example works ### Does this PR introduce _any_ user-facing change? doc-only change ### How was this patch tested? enabled doc-test ### Was this patch authored or co-authored using generative AI tooling? no Closes #52814 from zhengruifeng/py_test_pivot. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 2063c365860bb1620fdd4106ce227089d2691bf0) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/group.py | 35 +++++++++++++++++++---------------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 05021aabb50f8..939f7ff6b610c 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -456,7 +456,7 @@ def pivot(self, pivot_col: str, values: Optional[List["LiteralType"]] = None) -> Examples -------- - >>> from pyspark.sql import Row + >>> from pyspark.sql import Row, functions as sf >>> df1 = spark.createDataFrame([ ... Row(course="dotNET", year=2012, earnings=10000), ... Row(course="Java", year=2012, earnings=20000), @@ -474,28 +474,30 @@ def pivot(self, pivot_col: str, values: Optional[List["LiteralType"]] = None) -> |dotNET|2013| 48000| | Java|2013| 30000| +------+----+--------+ + >>> df2 = spark.createDataFrame([ ... Row(training="expert", sales=Row(course="dotNET", year=2012, earnings=10000)), ... Row(training="junior", sales=Row(course="Java", year=2012, earnings=20000)), ... Row(training="expert", sales=Row(course="dotNET", year=2012, earnings=5000)), ... Row(training="junior", sales=Row(course="dotNET", year=2013, earnings=48000)), ... Row(training="expert", sales=Row(course="Java", year=2013, earnings=30000)), - ... ]) # doctest: +SKIP - >>> df2.show() # doctest: +SKIP - +--------+--------------------+ - |training| sales| - +--------+--------------------+ - | expert|{dotNET, 2012, 10...| - | junior| {Java, 2012, 20000}| - | expert|{dotNET, 2012, 5000}| - | junior|{dotNET, 2013, 48...| - | expert| {Java, 2013, 30000}| - +--------+--------------------+ + ... ]) + >>> df2.show(truncate=False) + +--------+---------------------+ + |training|sales | + +--------+---------------------+ + |expert |{dotNET, 2012, 10000}| + |junior |{Java, 2012, 20000} | + |expert |{dotNET, 2012, 5000} | + |junior |{dotNET, 2013, 48000}| + |expert |{Java, 2013, 30000} | + +--------+---------------------+ Compute the sum of earnings for each year by course with each course as a separate column >>> df1.groupBy("year").pivot( - ... "course", ["dotNET", "Java"]).sum("earnings").sort("year").show() + ... "course", ["dotNET", "Java"] + ... ).sum("earnings").sort("year").show() +----+------+-----+ |year|dotNET| Java| +----+------+-----+ @@ -512,9 +514,10 @@ def pivot(self, pivot_col: str, values: Optional[List["LiteralType"]] = None) -> |2012|20000| 15000| |2013|30000| 48000| +----+-----+------+ - >>> df2.groupBy( - ... "sales.year").pivot("sales.course").sum("sales.earnings").sort("year").show() - ... # doctest: +SKIP + + >>> df2.groupBy("sales.year").pivot( + ... "sales.course" + ... ).agg(sf.sum("sales.earnings")).sort("year").show() +----+-----+------+ |year| Java|dotNET| +----+-----+------+ From a5fc89cbcab2bd1b17bf2ffbb94e1764844edd5c Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sun, 2 Nov 2025 20:27:29 -0800 Subject: [PATCH 009/400] [SPARK-54127][BUILD] Fix sbt inconsistent shading package ### What changes were proposed in this pull request? Fix sbt inconsistent shading package for `spark-connect` module ### Why are the changes needed? Take 4.1.0-preview3 as the example, which is built by Maven. ``` $ jar tf jars/spark-connect_2.13-4.1.0-preview3.jar | grep 'connect/grpc' | head org/sparkproject/connect/grpc/ org/sparkproject/connect/grpc/inprocess/ org/sparkproject/connect/grpc/inprocess/InProcessTransport$InProcessStream$InProcessServerStream.class org/sparkproject/connect/grpc/inprocess/InProcessTransport$2.class org/sparkproject/connect/grpc/inprocess/InProcessTransport$InProcessStream$InProcessClientStream.class org/sparkproject/connect/grpc/inprocess/InProcessTransport$SingleMessageProducer.class org/sparkproject/connect/grpc/inprocess/InProcessTransport$6.class org/sparkproject/connect/grpc/inprocess/InProcessTransport$1.class org/sparkproject/connect/grpc/inprocess/InProcessSocketAddress.class org/sparkproject/connect/grpc/inprocess/InProcessTransport$InProcessStream.class ``` the produced jar built by SBT has a different relocated package for grpc classes ``` $ jar tf jars/spark-connect_2.13-4.1.0-SNAPSHOT.jar | grep 'connect/grpc' | head org/sparkproject/connect/grpc/ org/sparkproject/connect/grpc/io/ org/sparkproject/connect/grpc/io/grpc/ org/sparkproject/connect/grpc/io/grpc/Attributes$1.class org/sparkproject/connect/grpc/io/grpc/Attributes$Builder.class org/sparkproject/connect/grpc/io/grpc/Attributes$Key.class org/sparkproject/connect/grpc/io/grpc/Attributes.class org/sparkproject/connect/grpc/io/grpc/BinaryLog.class org/sparkproject/connect/grpc/io/grpc/BindableService.class org/sparkproject/connect/grpc/io/grpc/CallCredentials$MetadataApplier.class ``` ### Does this PR introduce _any_ user-facing change? No, so far I don't see such inconsistent causes real issue. ### How was this patch tested? Verified the output of `dev/make-distribution.sh --sbt-enabled` ``` jar tf jars/spark-connect_2.13-4.1.0-SNAPSHOT.jar | grep 'connect/grpc' | head org/sparkproject/connect/grpc/ org/sparkproject/connect/grpc/Attributes$1.class org/sparkproject/connect/grpc/Attributes$Builder.class org/sparkproject/connect/grpc/Attributes$Key.class org/sparkproject/connect/grpc/Attributes.class org/sparkproject/connect/grpc/BinaryLog.class org/sparkproject/connect/grpc/BindableService.class org/sparkproject/connect/grpc/CallCredentials$MetadataApplier.class org/sparkproject/connect/grpc/CallCredentials$RequestInfo.class org/sparkproject/connect/grpc/CallCredentials.class ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52825 from pan3793/SPARK-54127. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 3d92bdf6ac1628dc72d525f8e16ba0790ee9f3ed) Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d852b4155bea9..1d8de063133e3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -808,7 +808,7 @@ object SparkConnect { }, (assembly / assemblyShadeRules) := Seq( - ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.grpc.@0").inAll, + ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.grpc.@1").inAll, ShadeRule.rename("com.google.common.**" -> "org.sparkproject.connect.guava.@1").inAll, ShadeRule.rename("com.google.thirdparty.**" -> "org.sparkproject.connect.guava.@1").inAll, ShadeRule.rename("com.google.protobuf.**" -> "org.sparkproject.connect.protobuf.@1").inAll, From 23afe0dccde58840d3c26d9b624573de1bb6417c Mon Sep 17 00:00:00 2001 From: Martin Grund Date: Sun, 2 Nov 2025 20:29:22 -0800 Subject: [PATCH 010/400] [SPARK-54128][CONNECT][SQL] Convert IllegalArgumentException to SparkException with proper error classes in Spark Connect server ### What changes were proposed in this pull request? This PR converts several IllegalArgumentException and UnsupportedOperationException instances in the Spark Connect server to proper SparkException with structured error classes, ensuring consistent error propagation to clients. Changes include: 1. Added error class definitions in error-conditions.json: - INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_TAG_REQUIRES_TAG - INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_OPERATION_ID_REQUIRES_ID - INVALID_PARAMETER_VALUE.STREAMING_LISTENER_COMMAND_MISSING - INVALID_ARTIFACT_PATH - UNSUPPORTED_FEATURE.INTERRUPT_TYPE 2. Updated service handlers: - SparkConnectInterruptHandler: Converted generic exceptions to SparkSQLException - SparkConnectAddArtifactsHandler: Converted to SparkRuntimeException for invalid paths - SparkConnectStreamingQueryListenerHandler: Converted to SparkSQLException 3. Added test coverage in SparkConnectServiceE2ESuite ### Why are the changes needed? Previously, the Spark Connect server threw generic Java exceptions that: - Did not include structured error classes - Could not be properly categorized by clients - Provided less actionable error information - Were inconsistent with Spark's error handling standards The error handling infrastructure (ErrorUtils.handleError) can only propagate error classes from SparkThrowable instances. Generic Java exceptions are converted to generic UNKNOWN errors, losing important context. ### Does this PR introduce _any_ user-facing change? Yes, but only to improve error messages. Before: java.lang.IllegalArgumentException: INTERRUPT_TYPE_TAG requested, but no operation_tag provided. After: [INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_TAG_REQUIRES_TAG] The value of parameter(s) operation_tag in interrupt is invalid: INTERRUPT_TYPE_TAG requested, but no operation_tag provided. Clients can now parse structured error classes for better error handling and recovery. ### How was this patch tested? 1. Added new test cases in SparkConnectServiceE2ESuite: - test("Interrupt with TAG type without operation_tag throws proper error class") - test("Interrupt with OPERATION_ID type without operation_id throws proper error class") 2. Manual verification of error class propagation through the gRPC layer 3. All modified files pass Scala style checks ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.5 Closes #52818 from grundprinzip/SPARK-error-messages. Authored-by: Martin Grund Signed-off-by: Dongjoon Hyun (cherry picked from commit a830dd251979513308b5bf3a99e086588ebfacc3) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 26 ++++++++++ ...ConnectStreamingQueryListenerHandler.scala | 6 ++- .../SparkConnectAddArtifactsHandler.scala | 6 +-- .../SparkConnectInterruptHandler.scala | 16 +++++-- .../service/AddArtifactsHandlerSuite.scala | 12 +---- .../service/SparkConnectServiceE2ESuite.scala | 48 +++++++++++++++++++ 6 files changed, 95 insertions(+), 19 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 34b72975cc077..c19e192d80eef 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2478,6 +2478,12 @@ ], "sqlState" : "22003" }, + "INVALID_ARTIFACT_PATH" : { + "message" : [ + "Artifact with name is invalid. The name must be a relative path and cannot reference parent/sibling/nephew directories." + ], + "sqlState" : "22023" + }, "INVALID_ATTRIBUTE_NAME_SYNTAX" : { "message" : [ "Syntax error in the attribute name: . Check that backticks appear in pairs, a quoted string is a complete name part and use a backtick only inside quoted name parts." @@ -3466,6 +3472,16 @@ "expects an integer literal, but got ." ] }, + "INTERRUPT_TYPE_OPERATION_ID_REQUIRES_ID" : { + "message" : [ + "INTERRUPT_TYPE_OPERATION_ID requested, but no operation_id provided." + ] + }, + "INTERRUPT_TYPE_TAG_REQUIRES_TAG" : { + "message" : [ + "INTERRUPT_TYPE_TAG requested, but no operation_tag provided." + ] + }, "LENGTH" : { "message" : [ "Expects `length` greater than or equal to 0, but got ." @@ -3496,6 +3512,11 @@ "Expects a positive or a negative value for `start`, but got 0." ] }, + "STREAMING_LISTENER_COMMAND_MISSING" : { + "message" : [ + "Missing command in StreamingQueryListenerBusCommand." + ] + }, "STRING" : { "message" : [ "expects a string literal, but got ." @@ -6352,6 +6373,11 @@ "INSERT INTO with IF NOT EXISTS in the PARTITION spec." ] }, + "INTERRUPT_TYPE" : { + "message" : [ + "Unsupported interrupt type: ." + ] + }, "LAMBDA_FUNCTION_WITH_PYTHON_UDF" : { "message" : [ "Lambda function with Python UDF in a higher order function." diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 04312a35a3b4b..7bee4539d01bd 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -21,6 +21,7 @@ import scala.util.control.NonFatal import io.grpc.stub.StreamObserver +import org.apache.spark.SparkSQLException import org.apache.spark.connect.proto.ExecutePlanResponse import org.apache.spark.connect.proto.StreamingQueryListenerBusCommand import org.apache.spark.connect.proto.StreamingQueryListenerEventsResult @@ -117,7 +118,10 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex return } case StreamingQueryListenerBusCommand.CommandCase.COMMAND_NOT_SET => - throw new IllegalArgumentException("Missing command in StreamingQueryListenerBusCommand") + throw new SparkSQLException( + errorClass = "INVALID_PARAMETER_VALUE.STREAMING_LISTENER_COMMAND_MISSING", + messageParameters = + Map("parameter" -> "command", "functionName" -> "StreamingQueryListenerBusCommand")) } executeHolder.eventsManager.postFinished() } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala index becd7d855133d..477d5b974facc 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala @@ -219,9 +219,9 @@ class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddAr ArtifactUtils.concatenatePaths(stagingDir, path) } catch { case _: IllegalArgumentException => - throw new IllegalArgumentException( - s"Artifact with name: $name is invalid. The `name` " + - s"must be a relative path and cannot reference parent/sibling/nephew directories.") + throw new SparkRuntimeException( + errorClass = "INVALID_ARTIFACT_PATH", + messageParameters = Map("name" -> name)) case NonFatal(e) => throw e } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterruptHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterruptHandler.scala index ae38e55d3c672..8f41257ccdfdb 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterruptHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterruptHandler.scala @@ -21,6 +21,7 @@ import scala.jdk.CollectionConverters._ import io.grpc.stub.StreamObserver +import org.apache.spark.SparkSQLException import org.apache.spark.connect.proto import org.apache.spark.internal.Logging @@ -41,18 +42,23 @@ class SparkConnectInterruptHandler(responseObserver: StreamObserver[proto.Interr sessionHolder.interruptAll() case proto.InterruptRequest.InterruptType.INTERRUPT_TYPE_TAG => if (!v.hasOperationTag) { - throw new IllegalArgumentException( - s"INTERRUPT_TYPE_TAG requested, but no operation_tag provided.") + throw new SparkSQLException( + errorClass = "INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_TAG_REQUIRES_TAG", + messageParameters = + Map("parameter" -> "operation_tag", "functionName" -> "interrupt")) } sessionHolder.interruptTag(v.getOperationTag) case proto.InterruptRequest.InterruptType.INTERRUPT_TYPE_OPERATION_ID => if (!v.hasOperationId) { - throw new IllegalArgumentException( - s"INTERRUPT_TYPE_OPERATION_ID requested, but no operation_id provided.") + throw new SparkSQLException( + errorClass = "INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_OPERATION_ID_REQUIRES_ID", + messageParameters = Map("parameter" -> "operation_id", "functionName" -> "interrupt")) } sessionHolder.interruptOperation(v.getOperationId) case other => - throw new UnsupportedOperationException(s"Unknown InterruptType $other!") + throw new SparkSQLException( + errorClass = "UNSUPPORTED_FEATURE.INTERRUPT_TYPE", + messageParameters = Map("interruptType" -> other.toString)) } val response = proto.InterruptResponse diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala index 6cc5daadfddd7..1df8ba46286cb 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala @@ -399,11 +399,7 @@ class AddArtifactsHandlerSuite extends SharedSparkSession with ResourceHelper { handler.onNext(req) } assert(e.getStatus.getCode == Code.INTERNAL) - val statusProto = StatusProto.fromThrowable(e) - assert(statusProto.getDetailsCount == 1) - val details = statusProto.getDetails(0) - val info = details.unpack(classOf[ErrorInfo]) - assert(info.getReason.contains("java.lang.IllegalArgumentException")) + assert(e.getMessage.contains("INVALID_ARTIFACT_PATH")) } handler.onCompleted() } finally { @@ -422,11 +418,7 @@ class AddArtifactsHandlerSuite extends SharedSparkSession with ResourceHelper { handler.onNext(req) } assert(e.getStatus.getCode == Code.INTERNAL) - val statusProto = StatusProto.fromThrowable(e) - assert(statusProto.getDetailsCount == 1) - val details = statusProto.getDetails(0) - val info = details.unpack(classOf[ErrorInfo]) - assert(info.getReason.contains("java.lang.IllegalArgumentException")) + assert(e.getMessage.contains("INVALID_ARTIFACT_PATH")) } handler.onCompleted() } finally { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala index 0e18ff711c4c5..6eee71db57093 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala @@ -317,4 +317,52 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { assert(error.getMessage.contains(fixedOperationId)) } } + + test("Interrupt with TAG type without operation_tag throws proper error class") { + withRawBlockingStub { stub => + // Create an interrupt request with INTERRUPT_TYPE_TAG but no operation_tag + val request = org.apache.spark.connect.proto.InterruptRequest + .newBuilder() + .setSessionId(UUID.randomUUID().toString) + .setUserContext(org.apache.spark.connect.proto.UserContext + .newBuilder() + .setUserId(defaultUserId)) + .setInterruptType( + org.apache.spark.connect.proto.InterruptRequest.InterruptType.INTERRUPT_TYPE_TAG) + .build() + + val error = intercept[io.grpc.StatusRuntimeException] { + stub.interrupt(request) + } + + // Verify the error is INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_TAG_REQUIRES_TAG + assert(error.getMessage.contains("INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_TAG_REQUIRES_TAG")) + assert(error.getMessage.contains("operation_tag")) + } + } + + test("Interrupt with OPERATION_ID type without operation_id throws proper error class") { + withRawBlockingStub { stub => + // Create an interrupt request with INTERRUPT_TYPE_OPERATION_ID but no operation_id + val request = org.apache.spark.connect.proto.InterruptRequest + .newBuilder() + .setSessionId(UUID.randomUUID().toString) + .setUserContext(org.apache.spark.connect.proto.UserContext + .newBuilder() + .setUserId(defaultUserId)) + .setInterruptType( + org.apache.spark.connect.proto.InterruptRequest.InterruptType.INTERRUPT_TYPE_OPERATION_ID) + .build() + + val error = intercept[io.grpc.StatusRuntimeException] { + stub.interrupt(request) + } + + // Verify the error is INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_OPERATION_ID_REQUIRES_ID + assert( + error.getMessage.contains( + "INVALID_PARAMETER_VALUE.INTERRUPT_TYPE_OPERATION_ID_REQUIRES_ID")) + assert(error.getMessage.contains("operation_id")) + } + } } From f0becdc6b71cd7957e1ca93a5e98e81acd1bcb0c Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Sun, 2 Nov 2025 20:32:19 -0800 Subject: [PATCH 011/400] [SPARK-54144][PYTHON] Short Circuit Eval Type Inferences ### What changes were proposed in this pull request? Short Circuit Eval Type Inferences: ### Why are the changes needed? minor optimization that avoid unnecessary inference ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #52843 from zhengruifeng/short_circuit_type_infer. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 6e4936d0fe8fef932c17a20260a227cdb32142eb) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/pandas/typehints.py | 26 +++++++++++++++++--------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/pandas/typehints.py b/python/pyspark/sql/pandas/typehints.py index 18858ab0cf686..7c95feee0cfea 100644 --- a/python/pyspark/sql/pandas/typehints.py +++ b/python/pyspark/sql/pandas/typehints.py @@ -353,6 +353,9 @@ def infer_group_arrow_eval_type( return_annotation, parameter_check_func=lambda t: t == pa.RecordBatch ) ) + if is_iterator_batch: + return PythonEvalType.SQL_GROUPED_MAP_ARROW_ITER_UDF + # Tuple[pa.Scalar, ...], Iterator[pa.RecordBatch] -> Iterator[pa.RecordBatch] is_iterator_batch_with_keys = ( len(parameters_sig) == 2 @@ -364,19 +367,21 @@ def infer_group_arrow_eval_type( return_annotation, parameter_check_func=lambda t: t == pa.RecordBatch ) ) - - if is_iterator_batch or is_iterator_batch_with_keys: + if is_iterator_batch_with_keys: return PythonEvalType.SQL_GROUPED_MAP_ARROW_ITER_UDF # pa.Table -> pa.Table is_table = ( len(parameters_sig) == 1 and parameters_sig[0] == pa.Table and return_annotation == pa.Table ) + if is_table: + return PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF + # Tuple[pa.Scalar, ...], pa.Table -> pa.Table is_table_with_keys = ( len(parameters_sig) == 2 and parameters_sig[1] == pa.Table and return_annotation == pa.Table ) - if is_table or is_table_with_keys: + if is_table_with_keys: return PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF return None @@ -441,6 +446,9 @@ def infer_group_pandas_eval_type( return_annotation, parameter_check_func=lambda t: t == pd.DataFrame ) ) + if is_iterator_dataframe: + return PythonEvalType.SQL_GROUPED_MAP_PANDAS_ITER_UDF + # Tuple[Any, ...], Iterator[pd.DataFrame] -> Iterator[pd.DataFrame] is_iterator_dataframe_with_keys = ( len(parameters_sig) == 2 @@ -452,8 +460,7 @@ def infer_group_pandas_eval_type( return_annotation, parameter_check_func=lambda t: t == pd.DataFrame ) ) - - if is_iterator_dataframe or is_iterator_dataframe_with_keys: + if is_iterator_dataframe_with_keys: return PythonEvalType.SQL_GROUPED_MAP_PANDAS_ITER_UDF # pd.DataFrame -> pd.DataFrame @@ -462,13 +469,16 @@ def infer_group_pandas_eval_type( and parameters_sig[0] == pd.DataFrame and return_annotation == pd.DataFrame ) + if is_dataframe: + return PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF + # Tuple[Any, ...], pd.DataFrame -> pd.DataFrame is_dataframe_with_keys = ( len(parameters_sig) == 2 and parameters_sig[1] == pd.DataFrame and return_annotation == pd.DataFrame ) - if is_dataframe or is_dataframe_with_keys: + if is_dataframe_with_keys: return PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF return None @@ -512,11 +522,9 @@ def check_iterator_annotation( def check_union_annotation( annotation: Any, parameter_check_func: Optional[Callable[[Any], bool]] = None ) -> bool: - import typing - # Note that we cannot rely on '__origin__' in other type hints as it has changed from version # to version. origin = getattr(annotation, "__origin__", None) - return origin == typing.Union and ( + return origin == Union and ( parameter_check_func is None or all(map(parameter_check_func, annotation.__args__)) ) From d29af1305696471956579c96beb34152620df7e1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 2 Nov 2025 21:05:25 -0800 Subject: [PATCH 012/400] [SPARK-54148][INFRA] Add `libwebp-dev` to all `dev/spark-test-image/*/Dockerfile` ### What changes were proposed in this pull request? This PR aims to add `libwebp-dev` to all `dev/spark-test-image/*/Dockerfile`. ### Why are the changes needed? Like we hit this issue before, this happens at all related docker images. - #52290 - #52838 I checked `dev` directory. - https://github.com/apache/spark/tree/branch-4.1/dev To put it simply consistent with `spark-rm/Dockerfile` behavior, we need this additionally 13 places because `libwebp-dev` is used only 3 times while `libtiff5-dev` is used 16 times. ``` $ git grep libtiff5-dev dev | wc -l 16 $ git grep libwebp-dev dev | wc -l 3 ``` ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52846 from dongjoon-hyun/SPARK-54148. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit f5bd5d09dfd3f6fff13ca97724f1766029f0e0eb) Signed-off-by: Dongjoon Hyun --- dev/spark-test-image/docs/Dockerfile | 1 + dev/spark-test-image/numpy-213/Dockerfile | 1 + dev/spark-test-image/pypy-310/Dockerfile | 1 + dev/spark-test-image/python-310/Dockerfile | 1 + dev/spark-test-image/python-311-classic-only/Dockerfile | 1 + dev/spark-test-image/python-311/Dockerfile | 1 + dev/spark-test-image/python-312/Dockerfile | 1 + dev/spark-test-image/python-313-nogil/Dockerfile | 1 + dev/spark-test-image/python-313/Dockerfile | 1 + dev/spark-test-image/python-314/Dockerfile | 1 + dev/spark-test-image/python-minimum/Dockerfile | 1 + dev/spark-test-image/python-ps-minimum/Dockerfile | 1 + dev/spark-test-image/sparkr/Dockerfile | 1 + 13 files changed, 13 insertions(+) diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index c4cd43b9eb3ba..4c1e68f724479 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ nodejs \ npm \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index 116154b663b07..d0409e61a51a6 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/pypy-310/Dockerfile b/dev/spark-test-image/pypy-310/Dockerfile index cddf0f8ea10a3..6f0b938bd1990 100644 --- a/dev/spark-test-image/pypy-310/Dockerfile +++ b/dev/spark-test-image/pypy-310/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index cfc03bccdf7c1..ef59237afee24 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-311-classic-only/Dockerfile b/dev/spark-test-image/python-311-classic-only/Dockerfile index 6a71317a5fe44..72f7ebec686e8 100644 --- a/dev/spark-test-image/python-311-classic-only/Dockerfile +++ b/dev/spark-test-image/python-311-classic-only/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 962f6427de6a8..25fd065753bd3 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index afa24025c46c7..82016bbec860f 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index c7d2faed010f1..7f608caab1933 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index dcc68575c496e..9fd53d233ac07 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 5ab4154dd0f71..0ba9b620bd8b0 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index 8f42d02023e50..122281ec0ea1d 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -50,6 +50,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 440fda96f0fc8..680697c3f2d74 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -50,6 +50,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ openjdk-17-jdk-headless \ pkg-config \ diff --git a/dev/spark-test-image/sparkr/Dockerfile b/dev/spark-test-image/sparkr/Dockerfile index 3312c0852bd77..6c0314c051d17 100644 --- a/dev/spark-test-image/sparkr/Dockerfile +++ b/dev/spark-test-image/sparkr/Dockerfile @@ -49,6 +49,7 @@ RUN apt-get update && apt-get install -y \ libpython3-dev \ libssl-dev \ libtiff5-dev \ + libwebp-dev \ libxml2-dev \ pandoc \ pkg-config \ From a88b15c98dd56ca3b44ed5edb7188448a0db319c Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 3 Nov 2025 14:34:56 +0800 Subject: [PATCH 013/400] [SPARK-54111][CONNECT] Support getCatalogs for SparkConnectDatabaseMetaData ### What changes were proposed in this pull request? Implement `getCatalogs` defined in `java.sql.DatabaseMetaData` for `SparkConnectDatabaseMetaData`. ```java /** * Retrieves the catalog names available in this database. The results * are ordered by catalog name. * *

The catalog column is: *

    *
  1. TABLE_CAT String {code =>} catalog name *
* * return a {code ResultSet} object in which each row has a * single {code String} column that is a catalog name * throws SQLException if a database access error occurs */ ResultSet getCatalogs() throws SQLException; ``` ### Why are the changes needed? Enhance API coverage of the Connect JDBC driver, for example, `get[Catalogs|Schemas|Tables|...]` APIs are used by SQL GUI tools such as DBeaver for displaying the tree category. ### Does this PR introduce _any_ user-facing change? No, the Connect JDBC driver is a new feature under development. ### How was this patch tested? New UT is added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52815 from pan3793/SPARK-54111. Authored-by: Cheng Pan Signed-off-by: yangjie01 (cherry picked from commit 2be1eb72e1ca27a9b882dec7dfea726c437821fa) Signed-off-by: yangjie01 --- .../jdbc/SparkConnectDatabaseMetaData.scala | 12 +++++- .../SparkConnectDatabaseMetaDataSuite.scala | 40 ++++++++++++++++++- 2 files changed, 48 insertions(+), 4 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala index a16cba5e3da4e..215c8256acbc3 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala @@ -25,6 +25,8 @@ import org.apache.spark.util.VersionUtils class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends DatabaseMetaData { + import conn.spark.implicits._ + override def allProceduresAreCallable: Boolean = false override def allTablesAreSelectable: Boolean = false @@ -288,8 +290,14 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas columnNamePattern: String): ResultSet = throw new SQLFeatureNotSupportedException - override def getCatalogs: ResultSet = - throw new SQLFeatureNotSupportedException + override def getCatalogs: ResultSet = { + conn.checkOpen() + + val df = conn.spark.sql("SHOW CATALOGS") + .select($"catalog".as("TABLE_CAT")) + .orderBy("TABLE_CAT") + new SparkConnectResultSet(df.collectResult()) + } override def getSchemas: ResultSet = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala index b2ecc163b2b8a..42596b56f4c56 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala @@ -19,16 +19,27 @@ package org.apache.spark.sql.connect.client.jdbc import java.sql.{Array => _, _} +import scala.util.Using + import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connect.client.jdbc.test.JdbcHelper -import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession} +import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession, SQLHelper} import org.apache.spark.util.VersionUtils class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSparkSession - with JdbcHelper { + with JdbcHelper with SQLHelper { def jdbcUrl: String = s"jdbc:sc://localhost:$serverPort" + // catalyst test jar is inaccessible here, but presents at the testing connect server classpath + private val TEST_IN_MEMORY_CATALOG = "org.apache.spark.sql.connector.catalog.InMemoryCatalog" + + private def registerCatalog( + name: String, className: String)(implicit spark: SparkSession): Unit = { + spark.conf.set(s"spark.sql.catalog.$name", className) + } + test("SparkConnectDatabaseMetaData simple methods") { withConnection { conn => val spark = conn.asInstanceOf[SparkConnectConnection].spark @@ -199,4 +210,29 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark // scalastyle:on line.size.limit } } + + test("SparkConnectDatabaseMetaData getCatalogs") { + withConnection { conn => + implicit val spark: SparkSession = conn.asInstanceOf[SparkConnectConnection].spark + + registerCatalog("testcat", TEST_IN_MEMORY_CATALOG) + registerCatalog("testcat2", TEST_IN_MEMORY_CATALOG) + + // forcibly initialize the registered catalogs because SHOW CATALOGS only + // returns the initialized catalogs. + spark.sql("USE testcat") + spark.sql("USE testcat2") + spark.sql("USE spark_catalog") + + val metadata = conn.getMetaData + Using.resource(metadata.getCatalogs) { rs => + val catalogs = new Iterator[String] { + def hasNext: Boolean = rs.next() + def next(): String = rs.getString("TABLE_CAT") + }.toSeq + // results are ordered by TABLE_CAT + assert(catalogs === Seq("spark_catalog", "testcat", "testcat2")) + } + } + } } From f8c2c36e328415acbccf9f189b2d0bb828ccc100 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 3 Nov 2025 16:24:14 +0800 Subject: [PATCH 014/400] [SPARK-54108][CONNECT] Revise execute* methods of SparkConnectStatement ### What changes were proposed in this pull request? This PR revises the following 3 `execute*` methods and one additional `getUpdateCount` method of `SparkConnectStatement` that are defined in `java.sql.Statement` ```java /** * Executes the given SQL statement, which returns a single * {code ResultSet} object. *

* Note:This method cannot be called on a * {code PreparedStatement} or {code CallableStatement}. * param sql an SQL statement to be sent to the database, typically a * static SQL {code SELECT} statement * return a {code ResultSet} object that contains the data produced * by the given query; never {code null} * throws SQLException if a database access error occurs, * this method is called on a closed {code Statement}, the given * SQL statement produces anything other than a single * {code ResultSet} object, the method is called on a * {code PreparedStatement} or {code CallableStatement} * throws SQLTimeoutException when the driver has determined that the * timeout value that was specified by the {code setQueryTimeout} * method has been exceeded and has at least attempted to cancel * the currently running {code Statement} */ ResultSet executeQuery(String sql) throws SQLException; /** * Executes the given SQL statement, which may be an {code INSERT}, * {code UPDATE}, or {code DELETE} statement or an * SQL statement that returns nothing, such as an SQL DDL statement. *

* Note:This method cannot be called on a * {code PreparedStatement} or {code CallableStatement}. * param sql an SQL Data Manipulation Language (DML) statement, such as {code INSERT}, {code UPDATE} or * {code DELETE}; or an SQL statement that returns nothing, * such as a DDL statement. * * return either (1) the row count for SQL Data Manipulation Language (DML) statements * or (2) 0 for SQL statements that return nothing * * throws SQLException if a database access error occurs, * this method is called on a closed {code Statement}, the given * SQL statement produces a {code ResultSet} object, the method is called on a * {code PreparedStatement} or {code CallableStatement} * throws SQLTimeoutException when the driver has determined that the * timeout value that was specified by the {code setQueryTimeout} * method has been exceeded and has at least attempted to cancel * the currently running {code Statement} */ int executeUpdate(String sql) throws SQLException; /** * Executes the given SQL statement, which may return multiple results. * In some (uncommon) situations, a single SQL statement may return * multiple result sets and/or update counts. Normally you can ignore * this unless you are (1) executing a stored procedure that you know may * return multiple results or (2) you are dynamically executing an * unknown SQL string. *

* The {code execute} method executes an SQL statement and indicates the * form of the first result. You must then use the methods * {code getResultSet} or {code getUpdateCount} * to retrieve the result, and {code getMoreResults} to * move to any subsequent result(s). *

*Note:This method cannot be called on a * {code PreparedStatement} or {code CallableStatement}. * param sql any SQL statement * return {code true} if the first result is a {code ResultSet} * object; {code false} if it is an update count or there are * no results * throws SQLException if a database access error occurs, * this method is called on a closed {code Statement}, * the method is called on a * {code PreparedStatement} or {code CallableStatement} * throws SQLTimeoutException when the driver has determined that the * timeout value that was specified by the {code setQueryTimeout} * method has been exceeded and has at least attempted to cancel * the currently running {code Statement} * see #getResultSet * see #getUpdateCount * see #getMoreResults */ boolean execute(String sql) throws SQLException; /** * Retrieves the current result as an update count; * if the result is a {code ResultSet} object or there are no more results, -1 * is returned. This method should be called only once per result. * * return the current result as an update count; -1 if the current result is a * {code ResultSet} object or there are no more results * throws SQLException if a database access error occurs or * this method is called on a closed {code Statement} * see #execute */ int getUpdateCount() throws SQLException; ``` ### Why are the changes needed? Make the implementation respect the JDBC API specification. ### Does this PR introduce _any_ user-facing change? No, Connect JDBC Driver is an unreleased feature. ### How was this patch tested? New UTs are added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52810 from pan3793/SPARK-54108. Authored-by: Cheng Pan Signed-off-by: yangjie01 (cherry picked from commit edfd3578d586ac6da019e49ca44727e3e8842cbd) Signed-off-by: yangjie01 --- .../client/jdbc/SparkConnectStatement.scala | 67 +++++++++++----- .../jdbc/SparkConnectStatementSuite.scala | 80 +++++++++++++++++++ 2 files changed, 127 insertions(+), 20 deletions(-) create mode 100644 sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala index 8de227f9d07c2..8b60f309ef6d3 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.connect.client.jdbc import java.sql.{Array => _, _} +import org.apache.spark.sql.connect.client.SparkResult + class SparkConnectStatement(conn: SparkConnectConnection) extends Statement { private var operationId: String = _ @@ -49,33 +51,51 @@ class SparkConnectStatement(conn: SparkConnectConnection) extends Statement { } override def executeQuery(sql: String): ResultSet = { - checkOpen() - - val df = conn.spark.sql(sql) - val sparkResult = df.collectResult() - operationId = sparkResult.operationId - resultSet = new SparkConnectResultSet(sparkResult, this) - resultSet + val hasResultSet = execute(sql) + if (hasResultSet) { + assert(resultSet != null) + resultSet + } else { + throw new SQLException("The query does not produce a ResultSet.") + } } override def executeUpdate(sql: String): Int = { - checkOpen() - - val df = conn.spark.sql(sql) - val sparkResult = df.collectResult() - operationId = sparkResult.operationId - resultSet = null + val hasResultSet = execute(sql) + if (hasResultSet) { + // user are not expected to access the result set in this case, + // we must close it to avoid memory leak. + resultSet.close() + throw new SQLException("The query produces a ResultSet.") + } else { + assert(resultSet == null) + getUpdateCount + } + } - // always return 0 because affected rows is not supported yet - 0 + private def hasResultSet(sparkResult: SparkResult[_]): Boolean = { + // suppose this works in most cases + sparkResult.schema.length > 0 } override def execute(sql: String): Boolean = { checkOpen() - // always perform executeQuery and reture a ResultSet - executeQuery(sql) - true + // stmt can be reused to execute more than one queries, + // reset before executing new query + operationId = null + resultSet = null + + val df = conn.spark.sql(sql) + val sparkResult = df.collectResult() + operationId = sparkResult.operationId + if (hasResultSet(sparkResult)) { + resultSet = new SparkConnectResultSet(sparkResult, this) + true + } else { + sparkResult.close() + false + } } override def getResultSet: ResultSet = { @@ -123,8 +143,15 @@ class SparkConnectStatement(conn: SparkConnectConnection) extends Statement { override def setCursorName(name: String): Unit = throw new SQLFeatureNotSupportedException - override def getUpdateCount: Int = - throw new SQLFeatureNotSupportedException + override def getUpdateCount: Int = { + checkOpen() + + if (resultSet != null) { + -1 + } else { + 0 // always return 0 because affected rows is not supported yet + } + } override def getMoreResults: Boolean = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala new file mode 100644 index 0000000000000..8e3b616372d89 --- /dev/null +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.client.jdbc + +import java.sql.{Array => _, _} + +import scala.util.Using + +import org.apache.spark.sql.connect.client.jdbc.test.JdbcHelper +import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession, SQLHelper} + +class SparkConnectStatementSuite extends ConnectFunSuite with RemoteSparkSession + with JdbcHelper with SQLHelper { + + override def jdbcUrl: String = s"jdbc:sc://localhost:$serverPort" + + test("returned result set and update count of execute* methods") { + withTable("t1", "t2", "t3") { + withStatement { stmt => + // CREATE TABLE + assert(!stmt.execute("CREATE TABLE t1 (id INT) USING Parquet")) + assert(stmt.getUpdateCount === 0) + assert(stmt.getResultSet === null) + + var se = intercept[SQLException] { + stmt.executeQuery("CREATE TABLE t2 (id INT) USING Parquet") + } + assert(se.getMessage === "The query does not produce a ResultSet.") + + assert(stmt.executeUpdate("CREATE TABLE t3 (id INT) USING Parquet") === 0) + assert(stmt.getResultSet === null) + + // INSERT INTO + assert(!stmt.execute("INSERT INTO t1 VALUES (1)")) + assert(stmt.getUpdateCount === 0) + assert(stmt.getResultSet === null) + + se = intercept[SQLException] { + stmt.executeQuery("INSERT INTO t1 VALUES (1)") + } + assert(se.getMessage === "The query does not produce a ResultSet.") + + assert(stmt.executeUpdate("INSERT INTO t1 VALUES (1)") === 0) + assert(stmt.getResultSet === null) + + // SELECT + assert(stmt.execute("SELECT id FROM t1")) + assert(stmt.getUpdateCount === -1) + Using.resource(stmt.getResultSet) { rs => + assert(rs !== null) + } + + Using.resource(stmt.executeQuery("SELECT id FROM t1")) { rs => + assert(stmt.getUpdateCount === -1) + assert(rs !== null) + } + + se = intercept[SQLException] { + stmt.executeUpdate("SELECT id FROM t1") + } + assert(se.getMessage === "The query produces a ResultSet.") + } + } + } +} From beedbda4a27f72bad07c64a0f69c4db3fa2cf5c9 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 3 Nov 2025 22:08:23 +0800 Subject: [PATCH 015/400] [SPARK-54146][CORE][SQL] Clean up the usage of deprecated Jackson API ### What changes were proposed in this pull request? This PR cleans up the usage of deprecated Jackson APIs as follows: ``` Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.ErrorClassesJsonReader.readAsMap.map, origin=com.fasterxml.jackson.databind.ObjectMapper.readValue Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.status.KVUtils.KVStoreScalaSerializer, origin=com.fasterxml.jackson.databind.ObjectMapper.setSerializationInclusion Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.status.api.v1.JacksonMessageWriter, origin=com.fasterxml.jackson.databind.ObjectMapper.setSerializationInclusion Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.SparkThrowableSuite, origin=com.fasterxml.jackson.databind.ObjectMapper.readValue Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.SparkThrowableSuite, origin=com.fasterxml.jackson.databind.ObjectMapper.setSerializationInclusion Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.SparkThrowableSuite, origin=com.fasterxml.jackson.databind.ObjectMapper.readValue Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.SparkThrowableSuite, origin=com.fasterxml.jackson.databind.ObjectMapper.readValue Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.catalyst.util.RebaseDateTime.loadRebaseRecords.jsonRebaseRecords, origin=com.fasterxml.jackson.module.scala.ClassTagExtensions.readValue, version=2.20.0 Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.catalyst.catalog.UserDefinedFunction.getObjectMapper, origin=com.fasterxml.jackson.databind.ObjectMapper.setSerializationInclusion Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.catalyst.catalog.ClusterBySpec.mapper, origin=com.fasterxml.jackson.databind.ObjectMapper.setSerializationInclusion Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.execution.streaming.state.RocksDBCheckpointMetadata.mapper, origin=com.fasterxml.jackson.databind.ObjectMapper.setSerializationInclusion ``` Relevant modifications reference: 1. Replace `T readValue(URL src, TypeReference valueTypeRef)` with `T readValue(InputStream src, TypeReference valueTypeRef)` https://github.com/FasterXML/jackson-databind/blob/4260f88180e5e45f3be1a290114e55c042bb2213/src/main/java/com/fasterxml/jackson/databind/ObjectMapper.java#L3848-L3877 ``` /** * Method to deserialize JSON content from given resource into given Java type. *

* NOTE: handling of {link java.net.URL} is delegated to * {link JsonFactory#createParser(java.net.URL)} and usually simply * calls {link java.net.URL#openStream()}, meaning no special handling * is done. If different HTTP connection options are needed you will need * to create {link java.io.InputStream} separately. * * throws IOException if a low-level I/O problem (unexpected end-of-input, * network error) occurs (passed through as-is without additional wrapping -- note * that this is one case where {link DeserializationFeature#WRAP_EXCEPTIONS} * does NOT result in wrapping of exception even if enabled) * throws StreamReadException if underlying input contains invalid content * of type {link JsonParser} supports (JSON for default case) * throws DatabindException if the input JSON structure does not match structure * expected for result type (or has other mismatch issues) * * deprecated since 2.20 deprecated as it calls {link JsonFactory#createParser(URL)}. * Instead, use equivalent methods that take InputStream inputs instead. */ Deprecated // since 2.20 SuppressWarnings("unchecked") public T readValue(URL src, Class valueType) throws IOException, StreamReadException, DatabindException { _assertNotNull("src", src); return (T) _readMapAndClose(_jsonFactory.createParser(src), _typeFactory.constructType(valueType)); } ``` 2. Replace `ObjectMapper setSerializationInclusion(JsonInclude.Include incl)` with `ObjectMapper setDefaultPropertyInclusion(JsonInclude.Value incl)` https://github.com/FasterXML/jackson-databind/blob/4260f88180e5e45f3be1a290114e55c042bb2213/src/main/java/com/fasterxml/jackson/databind/ObjectMapper.java#L1853-L1872 ``` /** * Convenience method, equivalent to calling: *

     *  setPropertyInclusion(JsonInclude.Value.construct(incl, incl));
     *
*

* NOTE: behavior differs slightly from 2.8, where second argument was * implied to be JsonInclude.Include.ALWAYS. *

* NOTE: in Jackson 3.x all configuration goes through {code ObjectMapper} builders, * see {link com.fasterxml.jackson.databind.cfg.MapperBuilder}, * and this method will be removed from 3.0. * * deprecated Since 2.9 use {link #setDefaultPropertyInclusion(JsonInclude.Include)} */ Deprecated public ObjectMapper setSerializationInclusion(JsonInclude.Include incl) { setPropertyInclusion(JsonInclude.Value.construct(incl, incl)); return this; } ``` ### Why are the changes needed? To clean up the usage of deprecated APIs and align with Jackson's recommended practices. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #52845 from LuciferYang/jackson-deprecation. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: yangjie01 (cherry picked from commit 46e7bfb58141bea9c51066d789bb228373977e9c) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/ErrorClassesJSONReader.scala | 2 +- .../src/main/scala/org/apache/spark/status/KVUtils.scala | 2 +- .../spark/status/api/v1/JacksonMessageWriter.scala | 2 +- .../scala/org/apache/spark/SparkThrowableSuite.scala | 9 +++++---- .../apache/spark/sql/catalyst/util/RebaseDateTime.scala | 2 +- .../spark/sql/catalyst/catalog/UserDefinedFunction.scala | 2 +- .../apache/spark/sql/catalyst/catalog/interface.scala | 2 +- .../execution/streaming/state/RocksDBFileManager.scala | 2 +- 8 files changed, 12 insertions(+), 11 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/ErrorClassesJSONReader.scala b/common/utils/src/main/scala/org/apache/spark/ErrorClassesJSONReader.scala index af76056dfe928..0d958e3f71604 100644 --- a/common/utils/src/main/scala/org/apache/spark/ErrorClassesJSONReader.scala +++ b/common/utils/src/main/scala/org/apache/spark/ErrorClassesJSONReader.scala @@ -152,7 +152,7 @@ private object ErrorClassesJsonReader { .addModule(DefaultScalaModule) .build() private def readAsMap(url: URL): Map[String, ErrorInfo] = { - val map = mapper.readValue(url, new TypeReference[Map[String, ErrorInfo]]() {}) + val map = mapper.readValue(url.openStream(), new TypeReference[Map[String, ErrorInfo]]() {}) val errorClassWithDots = map.collectFirst { case (errorClass, _) if errorClass.contains('.') => errorClass case (_, ErrorInfo(_, Some(map), _, _)) if map.keys.exists(_.contains('.')) => diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala index 76fb654f8da2d..1a9d36f6c6473 100644 --- a/core/src/main/scala/org/apache/spark/status/KVUtils.scala +++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala @@ -74,7 +74,7 @@ private[spark] object KVUtils extends Logging { private[spark] class KVStoreScalaSerializer extends KVStoreSerializer { mapper.registerModule(DefaultScalaModule) - mapper.setSerializationInclusion(JsonInclude.Include.NON_ABSENT) + mapper.setDefaultPropertyInclusion(JsonInclude.Include.NON_ABSENT) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 259d0aacc5755..1103f55297821 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -48,7 +48,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ } mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule) mapper.enable(SerializationFeature.INDENT_OUTPUT) - mapper.setSerializationInclusion(JsonInclude.Include.NON_ABSENT) + mapper.setDefaultPropertyInclusion(JsonInclude.Include.NON_ABSENT) mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat) override def isWriteable( diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index d182bd165f1f7..24a1fa7401752 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -75,7 +75,8 @@ class SparkThrowableSuite extends SparkFunSuite { .addModule(DefaultScalaModule) .enable(STRICT_DUPLICATE_DETECTION) .build() - mapper.readValue(errorJsonFilePath.toUri.toURL, new TypeReference[Map[String, ErrorInfo]]() {}) + mapper.readValue( + errorJsonFilePath.toUri.toURL.openStream(), new TypeReference[Map[String, ErrorInfo]]() {}) } test("Error conditions are correctly formatted") { @@ -88,7 +89,7 @@ class SparkThrowableSuite extends SparkFunSuite { val prettyPrinter = new DefaultPrettyPrinter() .withArrayIndenter(DefaultIndenter.SYSTEM_LINEFEED_INSTANCE) val rewrittenString = mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true) - .setSerializationInclusion(Include.NON_ABSENT) + .setDefaultPropertyInclusion(Include.NON_ABSENT) .writer(prettyPrinter) .writeValueAsString(errorReader.errorInfoMap) @@ -124,9 +125,9 @@ class SparkThrowableSuite extends SparkFunSuite { .enable(STRICT_DUPLICATE_DETECTION) .build() val errorClasses = mapper.readValue( - errorClassesJson, new TypeReference[Map[String, String]]() {}) + errorClassesJson.openStream(), new TypeReference[Map[String, String]]() {}) val errorStates = mapper.readValue( - errorStatesJson, new TypeReference[Map[String, ErrorStateInfo]]() {}) + errorStatesJson.openStream(), new TypeReference[Map[String, ErrorStateInfo]]() {}) val errorConditionStates = errorReader.errorInfoMap.values.toSeq.flatMap(_.sqlState).toSet assert(Set("22012", "22003", "42601").subsetOf(errorStates.keySet)) assert(errorClasses.keySet.filter(!_.matches("[A-Z0-9]{2}")).isEmpty) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 8dff1ceccfcfe..ca8e73a517d56 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -288,7 +288,7 @@ object RebaseDateTime { // `JsonRebaseRecord`. Mutable HashMap is used here instead of AnyRefMap due to SPARK-49491. private[sql] def loadRebaseRecords(fileName: String): HashMap[String, RebaseInfo] = { val file = SparkClassUtils.getSparkClassLoader.getResource(fileName) - val jsonRebaseRecords = mapper.readValue[Seq[JsonRebaseRecord]](file) + val jsonRebaseRecords = mapper.readValue[Seq[JsonRebaseRecord]](file.openStream()) val hashMap = new HashMap[String, RebaseInfo] hashMap.sizeHint(jsonRebaseRecords.size) jsonRebaseRecords.foreach { jsonRecord => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala index 8ed2414683522..3365b11b07424 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -132,7 +132,7 @@ object UserDefinedFunction { */ private def getObjectMapper: ObjectMapper = { val mapper = new ObjectMapper with ClassTagExtensions - mapper.setSerializationInclusion(Include.NON_ABSENT) + mapper.setDefaultPropertyInclusion(Include.NON_ABSENT) mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) mapper.registerModule(DefaultScalaModule) mapper diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 64d816587f4de..07d26813be940 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -278,7 +278,7 @@ case class ClusterBySpec(columnNames: Seq[NamedReference]) { object ClusterBySpec { private val mapper = { val ret = new ObjectMapper() with ClassTagExtensions - ret.setSerializationInclusion(Include.NON_ABSENT) + ret.setDefaultPropertyInclusion(Include.NON_ABSENT) ret.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) ret.registerModule(DefaultScalaModule) ret diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 7bef692e264a2..2e86ff70d58fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -1123,7 +1123,7 @@ object RocksDBCheckpointMetadata { /** Used to convert between classes and JSON. */ lazy val mapper = { val _mapper = new ObjectMapper with ClassTagExtensions - _mapper.setSerializationInclusion(Include.NON_ABSENT) + _mapper.setDefaultPropertyInclusion(Include.NON_ABSENT) _mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) _mapper.registerModule(DefaultScalaModule) _mapper From bc0f6f7a8a0db3d56b106f23956aa6e6e999d99d Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 3 Nov 2025 14:06:17 -0400 Subject: [PATCH 016/400] [SPARK-53525][CONNECT][FOLLOWUP] Spark Connect ArrowBatch Result Chunking - Scala Client ### What changes were proposed in this pull request? In the previous PR https://github.com/apache/spark/pull/52271 of Spark Connect ArrowBatch Result Chunking, both Server-side and PySpark client changes were implemented. In this PR, the corresponding Scala client changes are implemented, so large Arrow rows are now supported on the Scala client as well. To reproduce the existing issue we are solving here, run this code on Spark Connect Scala client: ``` val res = spark.sql("select repeat('a', 1024*1024*300)").collect() println(res(0).getString(0).length) ``` It fails with `RESOURCE_EXHAUSTED` error with message `gRPC message exceeds maximum size 134217728: 314573320`, because the server is trying to send an ExecutePlanResponse of ~300MB to the client. With the improvement introduced by the PR, the above code runs successfully and prints the expected result. ### Why are the changes needed? It improves Spark Connect stability when returning large rows. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52496 from xi-db/arrow-batch-chuking-scala-client. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell (cherry picked from commit daa83fc209da1aa920cc68b6dc75175b9f2252dc) Signed-off-by: Herman van Hovell --- .../sql/connect/ClientE2ETestSuite.scala | 159 +++++++++++++++++- .../sql/connect/test/RemoteSparkSession.scala | 26 ++- .../connect/client/SparkConnectClient.scala | 45 ++++- .../sql/connect/client/SparkResult.scala | 138 ++++++++++----- 4 files changed, 311 insertions(+), 57 deletions(-) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala index 8c336b6fa6d5d..450ff8ca62490 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala @@ -26,12 +26,14 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.{DurationInt, FiniteDuration} import scala.jdk.CollectionConverters._ +import io.grpc.{CallOptions, Channel, ClientCall, ClientInterceptor, ForwardingClientCall, ForwardingClientCallListener, MethodDescriptor} import org.apache.commons.io.output.TeeOutputStream import org.scalactic.TolerantNumerics import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkArithmeticException, SparkException, SparkUpgradeException} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} +import org.apache.spark.connect.proto import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.sql.{functions, AnalysisException, Observation, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, TableAlreadyExistsException, TempTableAlreadyExistsException} @@ -41,7 +43,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connect.ConnectConversions._ import org.apache.spark.sql.connect.client.{RetryPolicy, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.test.{ConnectFunSuite, IntegrationTestUtils, QueryTest, RemoteSparkSession, SQLHelper} -import org.apache.spark.sql.connect.test.SparkConnectServerUtils.port +import org.apache.spark.sql.connect.test.SparkConnectServerUtils.{createSparkSession, port} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ @@ -1848,6 +1850,161 @@ class ClientE2ETestSuite checkAnswer(df, Seq.empty) } } + + // Helper class to capture Arrow batch chunk information from gRPC responses + private class ArrowBatchInterceptor extends ClientInterceptor { + case class BatchInfo( + batchIndex: Int, + rowCount: Long, + startOffset: Long, + chunks: Seq[ChunkInfo]) { + def totalChunks: Int = chunks.length + } + + case class ChunkInfo( + batchIndex: Int, + chunkIndex: Int, + numChunksInBatch: Int, + rowCount: Long, + startOffset: Long, + dataSize: Int) + + private val batches: mutable.Buffer[BatchInfo] = mutable.Buffer.empty + private var currentBatchIndex: Int = 0 + private val currentBatchChunks: mutable.Buffer[ChunkInfo] = mutable.Buffer.empty + + override def interceptCall[ReqT, RespT]( + method: MethodDescriptor[ReqT, RespT], + callOptions: CallOptions, + next: Channel): ClientCall[ReqT, RespT] = { + new ForwardingClientCall.SimpleForwardingClientCall[ReqT, RespT]( + next.newCall(method, callOptions)) { + override def start( + responseListener: ClientCall.Listener[RespT], + headers: io.grpc.Metadata): Unit = { + super.start( + new ForwardingClientCallListener.SimpleForwardingClientCallListener[RespT]( + responseListener) { + override def onMessage(message: RespT): Unit = { + message match { + case response: proto.ExecutePlanResponse if response.hasArrowBatch => + val arrowBatch = response.getArrowBatch + // Track chunk information for every chunk + currentBatchChunks += ChunkInfo( + batchIndex = currentBatchIndex, + chunkIndex = arrowBatch.getChunkIndex.toInt, + numChunksInBatch = arrowBatch.getNumChunksInBatch.toInt, + rowCount = arrowBatch.getRowCount, + startOffset = arrowBatch.getStartOffset, + dataSize = arrowBatch.getData.size()) + // When we receive the last chunk, create the BatchInfo + if (currentBatchChunks.length == arrowBatch.getNumChunksInBatch) { + batches += BatchInfo( + batchIndex = currentBatchIndex, + rowCount = arrowBatch.getRowCount, + startOffset = arrowBatch.getStartOffset, + chunks = currentBatchChunks.toList) + currentBatchChunks.clear() + currentBatchIndex += 1 + } + case _ => // Not an ExecutePlanResponse with ArrowBatch, ignore + } + super.onMessage(message) + } + }, + headers) + } + } + } + + // Get all batch information + def getBatchInfos: Seq[BatchInfo] = batches.toSeq + + def clear(): Unit = { + currentBatchIndex = 0 + currentBatchChunks.clear() + batches.clear() + } + } + + test("Arrow batch result chunking") { + // This test validates that the client can correctly reassemble chunked Arrow batches + // using SequenceInputStream as implemented in SparkResult.processResponses + + // Two cases are tested here: + // (a) client preferred chunk size is set: the server should respect it + // (b) client preferred chunk size is not set: the server should use its own max chunk size + Seq((Some(1024), None), (None, Some(1024))).foreach { + case (preferredChunkSizeOpt, maxChunkSizeOpt) => + // Create interceptor to capture chunk information + val arrowBatchInterceptor = new ArrowBatchInterceptor() + + try { + // Set preferred chunk size if specified and add interceptor + preferredChunkSizeOpt match { + case Some(size) => + spark = createSparkSession( + _.preferredArrowChunkSize(Some(size)).interceptor(arrowBatchInterceptor)) + case None => + spark = createSparkSession(_.interceptor(arrowBatchInterceptor)) + } + // Set server max chunk size if specified + maxChunkSizeOpt.foreach { size => + spark.conf.set("spark.connect.session.resultChunking.maxChunkSize", size.toString) + } + + val sqlQuery = + "select id, CAST(id + 0.5 AS DOUBLE) as double_val from range(0, 2000, 1, 4)" + + // Execute the query using withResult to access SparkResult object + spark.sql(sqlQuery).withResult { result => + // Verify the results are correct and complete + assert(result.length == 2000) + + // Get batch information from interceptor + val batchInfos = arrowBatchInterceptor.getBatchInfos + + // Assert there are 4 batches (partitions) in total + assert(batchInfos.length == 4) + + // Validate chunk information for each batch + val maxChunkSize = preferredChunkSizeOpt.orElse(maxChunkSizeOpt).get + batchInfos.foreach { batch => + // In this example, the max chunk size is set to a small value, + // so each Arrow batch should be split into multiple chunks + assert(batch.totalChunks > 5) + assert(batch.chunks.nonEmpty) + assert(batch.chunks.length == batch.totalChunks) + batch.chunks.zipWithIndex.foreach { case (chunk, expectedIndex) => + assert(chunk.chunkIndex == expectedIndex) + assert(chunk.numChunksInBatch == batch.totalChunks) + assert(chunk.rowCount == batch.rowCount) + assert(chunk.startOffset == batch.startOffset) + assert(chunk.dataSize > 0) + assert(chunk.dataSize <= maxChunkSize) + } + } + + // Validate data integrity across the range to ensure chunking didn't corrupt anything + val rows = result.toArray + var expectedId = 0L + rows.foreach { row => + assert(row.getLong(0) == expectedId) + val expectedDouble = expectedId + 0.5 + val actualDouble = row.getDouble(1) + assert(math.abs(actualDouble - expectedDouble) < 0.001) + expectedId += 1 + } + } + } finally { + // Clean up configurations + maxChunkSizeOpt.foreach { _ => + spark.conf.unset("spark.connect.session.resultChunking.maxChunkSize") + } + arrowBatchInterceptor.clear() + } + } + } } private[sql] case class ClassData(a: String, b: Int) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala index efb6c721876c3..a239775a3a86b 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala @@ -187,19 +187,27 @@ object SparkConnectServerUtils { } def createSparkSession(): SparkSession = { + createSparkSession(identity) + } + + def createSparkSession( + customBuilderFunc: SparkConnectClient.Builder => SparkConnectClient.Builder) + : SparkSession = { SparkConnectServerUtils.start() + var builder = SparkConnectClient + .builder() + .userId("test") + .port(port) + .retryPolicy( + RetryPolicy + .defaultPolicy() + .copy(maxRetries = Some(10), maxBackoff = Some(FiniteDuration(30, "s")))) + + builder = customBuilderFunc(builder) val spark = SparkSession .builder() - .client( - SparkConnectClient - .builder() - .userId("test") - .port(port) - .retryPolicy(RetryPolicy - .defaultPolicy() - .copy(maxRetries = Some(10), maxBackoff = Some(FiniteDuration(30, "s")))) - .build()) + .client(builder.build()) .create() // Execute an RPC which will get retried until the server is up. diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index fa32eba91eb2c..e5fd16a7c2612 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -138,6 +138,22 @@ private[sql] class SparkConnectClient( .setSessionId(sessionId) .setClientType(userAgent) .addAllTags(tags.get.toSeq.asJava) + + // Add request option to allow result chunking. + if (configuration.allowArrowBatchChunking) { + val chunkingOptionsBuilder = proto.ResultChunkingOptions + .newBuilder() + .setAllowArrowBatchChunking(true) + configuration.preferredArrowChunkSize.foreach { size => + chunkingOptionsBuilder.setPreferredArrowChunkSize(size) + } + request.addRequestOptions( + proto.ExecutePlanRequest.RequestOption + .newBuilder() + .setResultChunkingOptions(chunkingOptionsBuilder.build()) + .build()) + } + serverSideSessionId.foreach(session => request.setClientObservedServerSideSessionId(session)) operationId.foreach { opId => require( @@ -332,6 +348,16 @@ private[sql] class SparkConnectClient( def copy(): SparkConnectClient = configuration.toSparkConnectClient + /** + * Returns whether arrow batch chunking is allowed. + */ + def allowArrowBatchChunking: Boolean = configuration.allowArrowBatchChunking + + /** + * Returns the preferred arrow chunk size in bytes. + */ + def preferredArrowChunkSize: Option[Int] = configuration.preferredArrowChunkSize + /** * Add a single artifact to the client session. * @@ -757,6 +783,21 @@ object SparkConnectClient { this } + def allowArrowBatchChunking(allow: Boolean): Builder = { + _configuration = _configuration.copy(allowArrowBatchChunking = allow) + this + } + + def allowArrowBatchChunking: Boolean = _configuration.allowArrowBatchChunking + + def preferredArrowChunkSize(size: Option[Int]): Builder = { + size.foreach(s => require(s > 0, "preferredArrowChunkSize must be positive")) + _configuration = _configuration.copy(preferredArrowChunkSize = size) + this + } + + def preferredArrowChunkSize: Option[Int] = _configuration.preferredArrowChunkSize + def build(): SparkConnectClient = _configuration.toSparkConnectClient } @@ -801,7 +842,9 @@ object SparkConnectClient { interceptors: List[ClientInterceptor] = List.empty, sessionId: Option[String] = None, grpcMaxMessageSize: Int = ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE, - grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) { + grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT, + allowArrowBatchChunking: Boolean = true, + preferredArrowChunkSize: Option[Int] = None) { private def isLocal = host.equals("localhost") diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index ef55edd10c8a3..43265e55a0ca9 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -16,18 +16,21 @@ */ package org.apache.spark.sql.connect.client +import java.io.SequenceInputStream import java.lang.ref.Cleaner import java.util.Objects import scala.collection.mutable import scala.jdk.CollectionConverters._ +import com.google.protobuf.ByteString import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch} import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics +import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} @@ -42,7 +45,8 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String) - extends AutoCloseable { self => + extends AutoCloseable + with Logging { self => case class StageInfo( stageId: Long, @@ -118,6 +122,7 @@ private[sql] class SparkResult[T]( stopOnFirstNonEmptyResponse: Boolean = false): Boolean = { var nonEmpty = false var stop = false + val arrowBatchChunksToAssemble = mutable.Buffer.empty[ByteString] while (!stop && responses.hasNext) { val response = responses.next() @@ -151,55 +156,96 @@ private[sql] class SparkResult[T]( stop |= stopOnSchema } if (response.hasArrowBatch) { - val ipcStreamBytes = response.getArrowBatch.getData - val expectedNumRows = response.getArrowBatch.getRowCount - val reader = new MessageIterator(ipcStreamBytes.newInput(), allocator) - if (arrowSchema == null) { - arrowSchema = reader.schema - stop |= stopOnArrowSchema - } else if (arrowSchema != reader.schema) { - throw new IllegalStateException( - s"""Schema Mismatch between expected and received schema: - |=== Expected Schema === - |$arrowSchema - |=== Received Schema === - |${reader.schema} - |""".stripMargin) - } - if (structType == null) { - // If the schema is not available yet, fallback to the arrow schema. - structType = ArrowUtils.fromArrowSchema(reader.schema) - } - if (response.getArrowBatch.hasStartOffset) { - val expectedStartOffset = response.getArrowBatch.getStartOffset - if (numRecords != expectedStartOffset) { + val arrowBatch = response.getArrowBatch + logDebug( + s"Received arrow batch rows=${arrowBatch.getRowCount} " + + s"Number of chunks in batch=${arrowBatch.getNumChunksInBatch} " + + s"Chunk index=${arrowBatch.getChunkIndex} " + + s"size=${arrowBatch.getData.size()}") + + if (arrowBatchChunksToAssemble.nonEmpty) { + // Expect next chunk of the same batch + if (arrowBatch.getChunkIndex != arrowBatchChunksToAssemble.size) { throw new IllegalStateException( - s"Expected arrow batch to start at row offset $numRecords in results, " + - s"but received arrow batch starting at offset $expectedStartOffset.") + s"Expected chunk index ${arrowBatchChunksToAssemble.size} of the " + + s"arrow batch but got ${arrowBatch.getChunkIndex}.") } - } - var numRecordsInBatch = 0 - val messages = Seq.newBuilder[ArrowMessage] - while (reader.hasNext) { - val message = reader.next() - message match { - case batch: ArrowRecordBatch => - numRecordsInBatch += batch.getLength - case _ => + } else { + // Expect next batch + if (arrowBatch.hasStartOffset) { + val expectedStartOffset = arrowBatch.getStartOffset + if (numRecords != expectedStartOffset) { + throw new IllegalStateException( + s"Expected arrow batch to start at row offset $numRecords in results, " + + s"but received arrow batch starting at offset $expectedStartOffset.") + } + } + if (arrowBatch.getChunkIndex != 0) { + throw new IllegalStateException( + s"Expected chunk index 0 of the next arrow batch " + + s"but got ${arrowBatch.getChunkIndex}.") } - messages += message - } - if (numRecordsInBatch != expectedNumRows) { - throw new IllegalStateException( - s"Expected $expectedNumRows rows in arrow batch but got $numRecordsInBatch.") } - // Skip the entire result if it is empty. - if (numRecordsInBatch > 0) { - numRecords += numRecordsInBatch - resultMap.put(nextResultIndex, (reader.bytesRead, messages.result())) - nextResultIndex += 1 - nonEmpty |= true - stop |= stopOnFirstNonEmptyResponse + + arrowBatchChunksToAssemble += arrowBatch.getData + + // Assemble the chunks to an arrow batch to process if + // (a) chunking is not enabled (numChunksInBatch is not set or is 0, + // in this case, it is the single chunk in the batch) + // (b) or the client has received all chunks of the batch. + if (!arrowBatch.hasNumChunksInBatch || + arrowBatch.getNumChunksInBatch == 0 || + arrowBatchChunksToAssemble.size == arrowBatch.getNumChunksInBatch) { + + val numChunks = arrowBatchChunksToAssemble.size + val inputStreams = + arrowBatchChunksToAssemble.map(_.newInput()).iterator.asJavaEnumeration + val input = new SequenceInputStream(inputStreams) + arrowBatchChunksToAssemble.clear() + logDebug(s"Assembling arrow batch from $numChunks chunks.") + + val expectedNumRows = arrowBatch.getRowCount + val reader = new MessageIterator(input, allocator) + if (arrowSchema == null) { + arrowSchema = reader.schema + stop |= stopOnArrowSchema + } else if (arrowSchema != reader.schema) { + throw new IllegalStateException( + s"""Schema Mismatch between expected and received schema: + |=== Expected Schema === + |$arrowSchema + |=== Received Schema === + |${reader.schema} + |""".stripMargin) + } + if (structType == null) { + // If the schema is not available yet, fallback to the arrow schema. + structType = ArrowUtils.fromArrowSchema(reader.schema) + } + + var numRecordsInBatch = 0 + val messages = Seq.newBuilder[ArrowMessage] + while (reader.hasNext) { + val message = reader.next() + message match { + case batch: ArrowRecordBatch => + numRecordsInBatch += batch.getLength + case _ => + } + messages += message + } + if (numRecordsInBatch != expectedNumRows) { + throw new IllegalStateException( + s"Expected $expectedNumRows rows in arrow batch but got $numRecordsInBatch.") + } + // Skip the entire result if it is empty. + if (numRecordsInBatch > 0) { + numRecords += numRecordsInBatch + resultMap.put(nextResultIndex, (reader.bytesRead, messages.result())) + nextResultIndex += 1 + nonEmpty |= true + stop |= stopOnFirstNonEmptyResponse + } } } } From f0d8acb7835a93b2e616933367d9ea5ce3ac33b5 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 4 Nov 2025 02:47:54 +0800 Subject: [PATCH 017/400] [SPARK-54132][SQL][TESTS] Cover HashedRelation#close in HashedRelationSuite ### What changes were proposed in this pull request? Add the following code in `HashedRelationSuite`, to cover the API `HashedRelation#close` in the test suite. ```scala protected override def afterEach(): Unit = { super.afterEach() assert(umm.executionMemoryUsed === 0) } ``` ### Why are the changes needed? Doing this will: 1. Ensure `HashedRelation#close` is called in test code, to lower memory footprint and avoid memory leak when executing tests. 2. Ensure implementations of `HashedRelation#close` free the allocated memory blocks correctly. It's an individual effort to improve the test quality, but also a prerequisite task for https://github.com/apache/spark/pull/52817. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? It's a test PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52830 from zhztheplayer/wip-54132. Authored-by: Hongze Zhang Signed-off-by: Wenchen Fan (cherry picked from commit a5e866f7ff57ef5f662d6b8882c680fcbcc5ac4a) Signed-off-by: Wenchen Fan --- .../execution/joins/HashedRelationSuite.scala | 90 ++++++++----------- 1 file changed, 39 insertions(+), 51 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 6da5e0b1a123e..b88a76bbfb575 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -40,14 +40,13 @@ import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.collection.CompactBuffer class HashedRelationSuite extends SharedSparkSession { + val umm = new UnifiedMemoryManager( + new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), + Long.MaxValue, + Long.MaxValue / 2, + 1) - val mm = new TaskMemoryManager( - new UnifiedMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, - 1), - 0) + val mm = new TaskMemoryManager(umm, 0) val rand = new Random(100) @@ -64,6 +63,11 @@ class HashedRelationSuite extends SharedSparkSession { val sparseRows = sparseArray.map(i => projection(InternalRow(i.toLong)).copy()) val randomRows = randomArray.map(i => projection(InternalRow(i.toLong)).copy()) + protected override def afterEach(): Unit = { + super.afterEach() + assert(umm.executionMemoryUsed === 0) + } + test("UnsafeHashedRelation") { val schema = StructType(StructField("a", IntegerType, true) :: Nil) val data = Array(InternalRow(0), InternalRow(1), InternalRow(2), InternalRow(2)) @@ -87,6 +91,7 @@ class HashedRelationSuite extends SharedSparkSession { val out = new ObjectOutputStream(os) hashed.asInstanceOf[UnsafeHashedRelation].writeExternal(out) out.flush() + hashed.close() val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) val hashed2 = new UnsafeHashedRelation() hashed2.readExternal(in) @@ -108,19 +113,13 @@ class HashedRelationSuite extends SharedSparkSession { } test("test serialization empty hash map") { - val taskMemoryManager = new TaskMemoryManager( - new UnifiedMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, - 1), - 0) - val binaryMap = new BytesToBytesMap(taskMemoryManager, 1, 1) + val binaryMap = new BytesToBytesMap(mm, 1, 1) val os = new ByteArrayOutputStream() val out = new ObjectOutputStream(os) val hashed = new UnsafeHashedRelation(1, 1, binaryMap) hashed.writeExternal(out) out.flush() + hashed.close() val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) val hashed2 = new UnsafeHashedRelation() hashed2.readExternal(in) @@ -149,9 +148,10 @@ class HashedRelationSuite extends SharedSparkSession { assert(row.getLong(0) === i) assert(row.getInt(1) === i + 1) } + longRelation.close() val longRelation2 = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) - .asInstanceOf[LongHashedRelation] + .asInstanceOf[LongHashedRelation] assert(!longRelation2.keyIsUnique) (0 until 100).foreach { i => val rows = longRelation2.get(i).toArray @@ -166,6 +166,7 @@ class HashedRelationSuite extends SharedSparkSession { val out = new ObjectOutputStream(os) longRelation2.writeExternal(out) out.flush() + longRelation2.close() val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) val relation = new LongHashedRelation() relation.readExternal(in) @@ -181,19 +182,12 @@ class HashedRelationSuite extends SharedSparkSession { } test("LongToUnsafeRowMap with very wide range") { - val taskMemoryManager = new TaskMemoryManager( - new UnifiedMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, - 1), - 0) val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) { // SPARK-16740 val keys = Seq(0L, Long.MaxValue, Long.MaxValue) - val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + val map = new LongToUnsafeRowMap(mm, 1) keys.foreach { k => map.append(k, unsafeProj(InternalRow(k))) } @@ -210,7 +204,7 @@ class HashedRelationSuite extends SharedSparkSession { { // SPARK-16802 val keys = Seq(Long.MaxValue, Long.MaxValue - 10) - val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + val map = new LongToUnsafeRowMap(mm, 1) keys.foreach { k => map.append(k, unsafeProj(InternalRow(k))) } @@ -226,20 +220,13 @@ class HashedRelationSuite extends SharedSparkSession { } test("LongToUnsafeRowMap with random keys") { - val taskMemoryManager = new TaskMemoryManager( - new UnifiedMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, - 1), - 0) val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) val N = 1000000 val rand = new Random val keys = (0 to N).map(x => rand.nextLong()).toArray - val map = new LongToUnsafeRowMap(taskMemoryManager, 10) + val map = new LongToUnsafeRowMap(mm, 10) keys.foreach { k => map.append(k, unsafeProj(InternalRow(k))) } @@ -249,8 +236,9 @@ class HashedRelationSuite extends SharedSparkSession { val out = new ObjectOutputStream(os) map.writeExternal(out) out.flush() + map.free() val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) - val map2 = new LongToUnsafeRowMap(taskMemoryManager, 1) + val map2 = new LongToUnsafeRowMap(mm, 1) map2.readExternal(in) val row = unsafeProj(InternalRow(0L)).copy() @@ -276,19 +264,12 @@ class HashedRelationSuite extends SharedSparkSession { } i += 1 } - map.free() + map2.free() } test("SPARK-24257: insert big values into LongToUnsafeRowMap") { - val taskMemoryManager = new TaskMemoryManager( - new UnifiedMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, - 1), - 0) val unsafeProj = UnsafeProjection.create(Array[DataType](StringType)) - val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + val map = new LongToUnsafeRowMap(mm, 1) val key = 0L // the page array is initialized with length 1 << 17 (1M bytes), @@ -343,6 +324,7 @@ class HashedRelationSuite extends SharedSparkSession { val rows = (0 until 100).map(i => unsafeProj(InternalRow(Int.int2long(i), i + 1)).copy()) val longRelation = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) val longRelation2 = ser.deserialize[LongHashedRelation](ser.serialize(longRelation)) + longRelation.close() (0 until 100).foreach { i => val rows = longRelation2.get(i).toArray assert(rows.length === 2) @@ -359,6 +341,7 @@ class HashedRelationSuite extends SharedSparkSession { unsafeHashed.asInstanceOf[UnsafeHashedRelation].writeExternal(out) out.flush() val unsafeHashed2 = ser.deserialize[UnsafeHashedRelation](ser.serialize(unsafeHashed)) + unsafeHashed.close() val os2 = new ByteArrayOutputStream() val out2 = new ObjectOutputStream(os2) unsafeHashed2.writeExternal(out2) @@ -398,6 +381,7 @@ class HashedRelationSuite extends SharedSparkSession { thread2.join() val unsafeHashed2 = ser.deserialize[UnsafeHashedRelation](ser.serialize(unsafeHashed)) + unsafeHashed.close() val os2 = new ByteArrayOutputStream() val out2 = new ObjectOutputStream(os2) unsafeHashed2.writeExternal(out2) @@ -452,18 +436,21 @@ class HashedRelationSuite extends SharedSparkSession { val hashedRelation = UnsafeHashedRelation(contiguousRows.iterator, singleKey, 1, mm) val keyIterator = hashedRelation.keys() assert(keyIterator.map(key => key.getLong(0)).toArray === contiguousArray) + hashedRelation.close() } test("UnsafeHashedRelation: key set iterator on a sparse array of keys") { val hashedRelation = UnsafeHashedRelation(sparseRows.iterator, singleKey, 1, mm) val keyIterator = hashedRelation.keys() assert(keyIterator.map(key => key.getLong(0)).toArray === sparseArray) + hashedRelation.close() } test("LongHashedRelation: key set iterator on a contiguous array of keys") { val longRelation = LongHashedRelation(contiguousRows.iterator, singleKey, 1, mm) val keyIterator = longRelation.keys() assert(keyIterator.map(key => key.getLong(0)).toArray === contiguousArray) + longRelation.close() } test("LongToUnsafeRowMap: key set iterator on a contiguous array of keys") { @@ -478,6 +465,7 @@ class HashedRelationSuite extends SharedSparkSession { rowMap.optimize() keyIterator = rowMap.keys() assert(keyIterator.map(key => key.getLong(0)).toArray === contiguousArray) + rowMap.free() } test("LongToUnsafeRowMap: key set iterator on a sparse array with equidistant keys") { @@ -490,6 +478,7 @@ class HashedRelationSuite extends SharedSparkSession { rowMap.optimize() keyIterator = rowMap.keys() assert(keyIterator.map(_.getLong(0)).toArray === sparseArray) + rowMap.free() } test("LongToUnsafeRowMap: key set iterator on an array with a single key") { @@ -530,6 +519,7 @@ class HashedRelationSuite extends SharedSparkSession { buffer.append(keyIterator.next().getLong(0)) } assert(buffer === randomArray) + rowMap.free() } test("LongToUnsafeRowMap: no explicit hasNext calls on the key iterator") { @@ -560,6 +550,7 @@ class HashedRelationSuite extends SharedSparkSession { buffer.append(keyIterator.next().getLong(0)) } assert(buffer === randomArray) + rowMap.free() } test("LongToUnsafeRowMap: call hasNext at the end of the iterator") { @@ -577,6 +568,7 @@ class HashedRelationSuite extends SharedSparkSession { assert(keyIterator.map(key => key.getLong(0)).toArray === sparseArray) assert(keyIterator.hasNext == false) assert(keyIterator.hasNext == false) + rowMap.free() } test("LongToUnsafeRowMap: random sequence of hasNext and next() calls on the key iterator") { @@ -607,6 +599,7 @@ class HashedRelationSuite extends SharedSparkSession { } } assert(buffer === randomArray) + rowMap.free() } test("HashJoin: packing and unpacking with the same key type in a LongType") { @@ -661,6 +654,7 @@ class HashedRelationSuite extends SharedSparkSession { assert(hashed.keys().isEmpty) assert(hashed.keyIsUnique) assert(hashed.estimatedSize == 0) + hashed.close() } test("SPARK-32399: test methods related to key index") { @@ -739,20 +733,14 @@ class HashedRelationSuite extends SharedSparkSession { val actualValues = row.map(_._2.getInt(1)) assert(actualValues === expectedValues) } + unsafeRelation.close() } test("LongToUnsafeRowMap support ignoresDuplicatedKey") { - val taskMemoryManager = new TaskMemoryManager( - new UnifiedMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, - 1), - 0) val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) val keys = Seq(1L, 1L, 1L) Seq(true, false).foreach { ignoresDuplicatedKey => - val map = new LongToUnsafeRowMap(taskMemoryManager, 1, ignoresDuplicatedKey) + val map = new LongToUnsafeRowMap(mm, 1, ignoresDuplicatedKey) keys.foreach { k => map.append(k, unsafeProj(InternalRow(k))) } From bd90d7168d6ee54de75e092ffa86fa06e67d3725 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Tue, 4 Nov 2025 02:51:30 +0800 Subject: [PATCH 018/400] [SPARK-54110][GEO][SQL] Introduce type encoders for Geography and Geometry types ### What changes were proposed in this pull request? This PR introduces type encoders for `Geography` and `Geometry`. Note that the server-side geospatial classes have already been introduced as part of: https://github.com/apache/spark/pull/52737; while client-side geospatial classes in external API have subsequently been introduced as part of: https://github.com/apache/spark/pull/52804. ### Why are the changes needed? These encoders are used to translate between (server) Spark Catalyst types and (client) Java/Scala types. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new Scala unit test suites for data frames: - `GeographyDataFrameSuite` - `GeometryDataFrameSuite` Also, added appropriate test cases to: - `RowSuite` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52813 from uros-db/geo-expression-encoders. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit 2f9431340937d4b2a6cde54cb35b780d6c03b512) Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 6 + .../scala/org/apache/spark/sql/Encoders.scala | 14 ++ .../main/scala/org/apache/spark/sql/Row.scala | 18 ++ .../org/apache/spark/sql/SQLImplicits.scala | 8 + .../sql/catalyst/JavaTypeInference.scala | 6 +- .../spark/sql/catalyst/ScalaReflection.scala | 4 + .../catalyst/encoders/AgnosticEncoder.scala | 6 + .../sql/catalyst/encoders/RowEncoder.scala | 4 +- .../spark/sql/types/GeographyType.scala | 28 ++- .../apache/spark/sql/types/GeometryType.scala | 28 ++- .../spark/sql/catalyst/util/STUtils.java | 34 ++++ .../sql/catalyst/CatalystTypeConverters.scala | 42 +++- .../catalyst/DeserializerBuildHelper.scala | 26 ++- .../sql/catalyst/SerializerBuildHelper.scala | 24 ++- .../spark/sql/GeographyDataFrameSuite.scala | 180 +++++++++++++++++ .../spark/sql/GeometryDataFrameSuite.scala | 181 ++++++++++++++++++ .../scala/org/apache/spark/sql/RowSuite.scala | 11 ++ 17 files changed, 611 insertions(+), 9 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index c19e192d80eef..9d95d74cc21a1 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1888,6 +1888,12 @@ ], "sqlState" : "42623" }, + "GEO_ENCODER_SRID_MISMATCH_ERROR" : { + "message" : [ + "Failed to encode value because provided SRID of a value to encode does not match type SRID: ." + ], + "sqlState" : "42K09" + }, "GET_TABLES_BY_TYPE_UNSUPPORTED_BY_HIVE_VERSION" : { "message" : [ "Hive 2.2 and lower versions don't support getTablesByType. Please use Hive 2.3 or higher version." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala index cb1402e1b0f4a..7e698e58321ee 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -162,6 +162,20 @@ object Encoders { */ def BINARY: Encoder[Array[Byte]] = BinaryEncoder + /** + * An encoder for Geometry data type. + * + * @since 4.1.0 + */ + def GEOMETRY(dt: GeometryType): Encoder[Geometry] = GeometryEncoder(dt) + + /** + * An encoder for Geography data type. + * + * @since 4.1.0 + */ + def GEOGRAPHY(dt: GeographyType): Encoder[Geography] = GeographyEncoder(dt) + /** * Creates an encoder that serializes instances of the `java.time.Duration` class to the * internal representation of nullable Catalyst's DayTimeIntervalType. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala index 764bdb17b37e2..1019d4c9a2276 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala @@ -302,6 +302,24 @@ trait Row extends Serializable { */ def getDecimal(i: Int): java.math.BigDecimal = getAs[java.math.BigDecimal](i) + /** + * Returns the value at position i of date type as org.apache.spark.sql.types.Geometry. + * + * @throws ClassCastException + * when data type does not match. + */ + def getGeometry(i: Int): org.apache.spark.sql.types.Geometry = + getAs[org.apache.spark.sql.types.Geometry](i) + + /** + * Returns the value at position i of date type as org.apache.spark.sql.types.Geography. + * + * @throws ClassCastException + * when data type does not match. + */ + def getGeography(i: Int): org.apache.spark.sql.types.Geography = + getAs[org.apache.spark.sql.types.Geography](i) + /** * Returns the value at position i of date type as java.sql.Date. * diff --git a/sql/api/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/api/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index a5b1060ca03db..9d64225b96633 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -104,6 +104,14 @@ trait EncoderImplicits extends LowPrioritySQLImplicits with Serializable { implicit def newScalaDecimalEncoder: Encoder[scala.math.BigDecimal] = DEFAULT_SCALA_DECIMAL_ENCODER + /** @since 4.1.0 */ + implicit def newGeometryEncoder: Encoder[org.apache.spark.sql.types.Geometry] = + DEFAULT_GEOMETRY_ENCODER + + /** @since 4.1.0 */ + implicit def newGeographyEncoder: Encoder[org.apache.spark.sql.types.Geography] = + DEFAULT_GEOGRAPHY_ENCODER + /** @since 2.2.0 */ implicit def newDateEncoder: Encoder[java.sql.Date] = Encoders.DATE diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 906e6419b3607..91947cf416fb6 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.commons.lang3.reflect.{TypeUtils => JavaTypeUtils} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, DayTimeIntervalEncoder, DEFAULT_JAVA_DECIMAL_ENCODER, EncoderField, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaEnumEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, STRICT_DATE_ENCODER, STRICT_INSTANT_ENCODER, STRICT_LOCAL_DATE_ENCODER, STRICT_TIMESTAMP_ENCODER, StringEncoder, UDTEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, DayTimeIntervalEncoder, DEFAULT_GEOGRAPHY_ENCODER, DEFAULT_GEOMETRY_ENCODER, DEFAULT_JAVA_DECIMAL_ENCODER, EncoderField, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaEnumEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, STRICT_DATE_ENCODER, STRICT_INSTANT_ENCODER, STRICT_LOCAL_DATE_ENCODER, STRICT_TIMESTAMP_ENCODER, StringEncoder, UDTEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.errors.ExecutionErrors import org.apache.spark.sql.types._ import org.apache.spark.util.ArrayImplicits._ @@ -86,6 +86,10 @@ object JavaTypeInference { case c: Class[_] if c == classOf[java.lang.String] => StringEncoder case c: Class[_] if c == classOf[Array[Byte]] => BinaryEncoder + case c: Class[_] if c == classOf[org.apache.spark.sql.types.Geometry] => + DEFAULT_GEOMETRY_ENCODER + case c: Class[_] if c == classOf[org.apache.spark.sql.types.Geography] => + DEFAULT_GEOGRAPHY_ENCODER case c: Class[_] if c == classOf[java.math.BigDecimal] => DEFAULT_JAVA_DECIMAL_ENCODER case c: Class[_] if c == classOf[java.math.BigInteger] => JavaBigIntEncoder case c: Class[_] if c == classOf[java.time.LocalDate] => STRICT_LOCAL_DATE_ENCODER diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index d2e0053597e4f..6f5c4be42bbd4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -332,6 +332,10 @@ object ScalaReflection extends ScalaReflection { case t if isSubtype(t, localTypeOf[java.time.LocalDateTime]) => LocalDateTimeEncoder case t if isSubtype(t, localTypeOf[java.time.LocalTime]) => LocalTimeEncoder case t if isSubtype(t, localTypeOf[VariantVal]) => VariantEncoder + case t if isSubtype(t, localTypeOf[Geography]) => + DEFAULT_GEOGRAPHY_ENCODER + case t if isSubtype(t, localTypeOf[Geometry]) => + DEFAULT_GEOMETRY_ENCODER case t if isSubtype(t, localTypeOf[Row]) => UnboundRowEncoder // UDT encoders diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala index 0c5295176608f..20949c188cb81 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala @@ -246,6 +246,8 @@ object AgnosticEncoders { case object DayTimeIntervalEncoder extends LeafEncoder[Duration](DayTimeIntervalType()) case object YearMonthIntervalEncoder extends LeafEncoder[Period](YearMonthIntervalType()) case object VariantEncoder extends LeafEncoder[VariantVal](VariantType) + case class GeographyEncoder(dt: GeographyType) extends LeafEncoder[Geography](dt) + case class GeometryEncoder(dt: GeometryType) extends LeafEncoder[Geometry](dt) case class DateEncoder(override val lenientSerialization: Boolean) extends LeafEncoder[jsql.Date](DateType) case class LocalDateEncoder(override val lenientSerialization: Boolean) @@ -277,6 +279,10 @@ object AgnosticEncoders { ScalaDecimalEncoder(DecimalType.SYSTEM_DEFAULT) val DEFAULT_JAVA_DECIMAL_ENCODER: JavaDecimalEncoder = JavaDecimalEncoder(DecimalType.SYSTEM_DEFAULT, lenientSerialization = false) + val DEFAULT_GEOMETRY_ENCODER: GeometryEncoder = + GeometryEncoder(GeometryType(Geometry.DEFAULT_SRID)) + val DEFAULT_GEOGRAPHY_ENCODER: GeographyEncoder = + GeographyEncoder(GeographyType(Geography.DEFAULT_SRID)) /** * Encoder that transforms external data into a representation that can be further processed by diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 620278c66d21d..73152017cf225 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.reflect.classTag import org.apache.spark.sql.{AnalysisException, Row} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, NullEncoder, RowEncoder => AgnosticRowEncoder, StringEncoder, TimestampEncoder, UDTEncoder, VarcharEncoder, VariantEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, GeographyEncoder, GeometryEncoder, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, NullEncoder, RowEncoder => AgnosticRowEncoder, StringEncoder, TimestampEncoder, UDTEncoder, VarcharEncoder, VariantEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.errors.DataTypeErrorsBase import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ @@ -120,6 +120,8 @@ object RowEncoder extends DataTypeErrorsBase { field.nullable, field.metadata) }.toImmutableArraySeq) + case g: GeographyType => GeographyEncoder(g) + case g: GeometryType => GeometryEncoder(g) case _ => throw new AnalysisException( diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala index 638ae79351846..d72e5987abebd 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.{JString, JValue} -import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.internal.types.GeographicSpatialReferenceSystemMapper @@ -133,6 +133,27 @@ class GeographyType private (val crs: String, val algorithm: EdgeInterpolationAl // If the SRID is not mixed, we can only accept the same SRID. isMixedSrid || gt.srid == srid } + + private[sql] def assertSridAllowedForType(otherSrid: Int): Unit = { + // If SRID is not mixed, SRIDs must match. + if (!isMixedSrid && otherSrid != srid) { + throw new SparkRuntimeException( + errorClass = "GEO_ENCODER_SRID_MISMATCH_ERROR", + messageParameters = Map( + "type" -> "GEOGRAPHY", + "valueSrid" -> otherSrid.toString, + "typeSrid" -> srid.toString)) + } else if (isMixedSrid) { + // For fixed SRID geom types, we have a check that value matches the type srid. + // For mixed SRID we need to do that check explicitly, as MIXED SRID can accept any SRID. + // However it should accept only valid SRIDs. + if (!GeographyType.isSridSupported(otherSrid)) { + throw new SparkIllegalArgumentException( + errorClass = "ST_INVALID_SRID_VALUE", + messageParameters = Map("srid" -> otherSrid.toString)) + } + } + } } @Experimental @@ -157,6 +178,11 @@ object GeographyType extends SpatialType { private final val GEOGRAPHY_MIXED_TYPE: GeographyType = GeographyType(MIXED_CRS, GEOGRAPHY_DEFAULT_ALGORITHM) + /** Returns whether the given SRID is supported. */ + private[types] def isSridSupported(srid: Int): Boolean = { + GeographicSpatialReferenceSystemMapper.getStringId(srid) != null + } + /** * Constructors for GeographyType. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala index 77a6b365c042a..f5bbbcba6706e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.{JString, JValue} -import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.internal.types.CartesianSpatialReferenceSystemMapper @@ -130,6 +130,27 @@ class GeometryType private (val crs: String) extends AtomicType with Serializabl // If the SRID is not mixed, we can only accept the same SRID. isMixedSrid || gt.srid == srid } + + private[sql] def assertSridAllowedForType(otherSrid: Int): Unit = { + // If SRID is not mixed, SRIDs must match. + if (!isMixedSrid && otherSrid != srid) { + throw new SparkRuntimeException( + errorClass = "GEO_ENCODER_SRID_MISMATCH_ERROR", + messageParameters = Map( + "type" -> "GEOMETRY", + "valueSrid" -> otherSrid.toString, + "typeSrid" -> srid.toString)) + } else if (isMixedSrid) { + // For fixed SRID geom types, we have a check that value matches the type srid. + // For mixed SRID we need to do that check explicitly, as MIXED SRID can accept any SRID. + // However it should accept only valid SRIDs. + if (!GeometryType.isSridSupported(otherSrid)) { + throw new SparkIllegalArgumentException( + errorClass = "ST_INVALID_SRID_VALUE", + messageParameters = Map("srid" -> otherSrid.toString)) + } + } + } } @Experimental @@ -149,6 +170,11 @@ object GeometryType extends SpatialType { private final val GEOMETRY_MIXED_TYPE: GeometryType = GeometryType(MIXED_CRS) + /** Returns whether the given SRID is supported. */ + private[types] def isSridSupported(srid: Int): Boolean = { + CartesianSpatialReferenceSystemMapper.getStringId(srid) != null + } + /** * Constructors for GeometryType. */ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java index aca3fdf1f1000..9edeee26eb98a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.catalyst.util; +import org.apache.spark.sql.types.GeographyType; +import org.apache.spark.sql.types.GeometryType; import org.apache.spark.unsafe.types.GeographyVal; import org.apache.spark.unsafe.types.GeometryVal; @@ -46,6 +48,38 @@ static GeometryVal toPhysVal(Geometry g) { return g.getValue(); } + /** Geospatial type encoder/decoder utilities. */ + + public static GeometryVal serializeGeomFromWKB(org.apache.spark.sql.types.Geometry geometry, + GeometryType gt) { + int geometrySrid = geometry.getSrid(); + gt.assertSridAllowedForType(geometrySrid); + return toPhysVal(Geometry.fromWkb(geometry.getBytes(), geometrySrid)); + } + + public static GeographyVal serializeGeogFromWKB(org.apache.spark.sql.types.Geography geography, + GeographyType gt) { + int geographySrid = geography.getSrid(); + gt.assertSridAllowedForType(geographySrid); + return toPhysVal(Geography.fromWkb(geography.getBytes(), geographySrid)); + } + + public static org.apache.spark.sql.types.Geometry deserializeGeom( + GeometryVal geometry, GeometryType gt) { + int geometrySrid = stSrid(geometry); + gt.assertSridAllowedForType(geometrySrid); + byte[] wkb = stAsBinary(geometry); + return org.apache.spark.sql.types.Geometry.fromWKB(wkb, geometrySrid); + } + + public static org.apache.spark.sql.types.Geography deserializeGeog( + GeographyVal geography, GeographyType gt) { + int geographySrid = stSrid(geography); + gt.assertSridAllowedForType(geographySrid); + byte[] wkb = stAsBinary(geography); + return org.apache.spark.sql.types.Geography.fromWKB(wkb, geographySrid); + } + /** Methods for implementing ST expressions. */ // ST_AsBinary diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index c1e0674d391d2..b8eee5e1c7c6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DayTimeIntervalType._ import org.apache.spark.sql.types.YearMonthIntervalType._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{GeographyVal, GeometryVal, UTF8String} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.collection.Utils @@ -69,6 +69,10 @@ object CatalystTypeConverters { case CharType(length) => new CharConverter(length) case VarcharType(length) => new VarcharConverter(length) case _: StringType => StringConverter + case g: GeographyType => + new GeographyConverter(g) + case g: GeometryType => + new GeometryConverter(g) case DateType if SQLConf.get.datetimeJava8ApiEnabled => LocalDateConverter case DateType => DateConverter case _: TimeType => TimeConverter @@ -345,6 +349,42 @@ object CatalystTypeConverters { row.getUTF8String(column).toString } + private class GeometryConverter(dataType: GeometryType) + extends CatalystTypeConverter[Any, org.apache.spark.sql.types.Geometry, GeometryVal] { + override def toCatalystImpl(scalaValue: Any): GeometryVal = scalaValue match { + case g: org.apache.spark.sql.types.Geometry => STUtils.serializeGeomFromWKB(g, dataType) + case other => throw new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_3219", + messageParameters = scala.collection.immutable.Map( + "other" -> other.toString, + "otherClass" -> other.getClass.getCanonicalName, + "dataType" -> StringType.sql)) + } + override def toScala(catalystValue: GeometryVal): org.apache.spark.sql.types.Geometry = + if (catalystValue == null) null + else STUtils.deserializeGeom(catalystValue, dataType) + override def toScalaImpl(row: InternalRow, column: Int): org.apache.spark.sql.types.Geometry = + STUtils.deserializeGeom(row.getGeometry(0), dataType) + } + + private class GeographyConverter(dataType: GeographyType) + extends CatalystTypeConverter[Any, org.apache.spark.sql.types.Geography, GeographyVal] { + override def toCatalystImpl(scalaValue: Any): GeographyVal = scalaValue match { + case g: org.apache.spark.sql.types.Geography => STUtils.serializeGeogFromWKB(g, dataType) + case other => throw new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_3219", + messageParameters = scala.collection.immutable.Map( + "other" -> other.toString, + "otherClass" -> other.getClass.getCanonicalName, + "dataType" -> StringType.sql)) + } + override def toScala(catalystValue: GeographyVal): org.apache.spark.sql.types.Geography = + if (catalystValue == null) null + else STUtils.deserializeGeog(catalystValue, dataType) + override def toScalaImpl(row: InternalRow, column: Int): org.apache.spark.sql.types.Geography = + STUtils.deserializeGeog(row.getGeography(0), dataType) + } + private object DateConverter extends CatalystTypeConverter[Any, Date, Any] { override def toCatalystImpl(scalaValue: Any): Int = scalaValue match { case d: Date => DateTimeUtils.fromJavaDate(d) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index a051205829a11..60de179edb799 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.{expressions => exprs} import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders, AgnosticExpressionPathEncoder, Codec, JavaSerializationCodec, KryoSerializationCodec} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, VarcharEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, GeographyEncoder, GeometryEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, VarcharEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{dataTypeForClass, externalDataTypeFor, isNativeEncoder} import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, IsNull, Literal, MapKeys, MapValues, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, DecodeUsingSerializer, InitializeJavaBean, Invoke, NewInstance, StaticInvoke, UnresolvedCatalystToExternalMap, UnresolvedMapObjects, WrapOption} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CharVarcharCodegenUtils, DateTimeUtils, IntervalUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CharVarcharCodegenUtils, DateTimeUtils, IntervalUtils, STUtils} import org.apache.spark.sql.types._ object DeserializerBuildHelper { @@ -80,6 +80,24 @@ object DeserializerBuildHelper { returnNullable = false) } + def createDeserializerForGeometryType(inputObject: Expression, gt: GeometryType): Expression = { + StaticInvoke( + classOf[STUtils], + ObjectType(classOf[Geometry]), + "deserializeGeom", + inputObject :: Literal.fromObject(gt) :: Nil, + returnNullable = false) + } + + def createDeserializerForGeographyType(inputObject: Expression, gt: GeographyType): Expression = { + StaticInvoke( + classOf[STUtils], + ObjectType(classOf[Geography]), + "deserializeGeog", + inputObject :: Literal.fromObject(gt) :: Nil, + returnNullable = false) + } + def createDeserializerForChar( path: Expression, returnNullable: Boolean, @@ -290,6 +308,10 @@ object DeserializerBuildHelper { "withName", createDeserializerForString(path, returnNullable = false) :: Nil, returnNullable = false) + case g: GeographyEncoder => + createDeserializerForGeographyType(path, g.dt) + case g: GeometryEncoder => + createDeserializerForGeometryType(path, g.dt) case CharEncoder(length) => createDeserializerForChar(path, returnNullable = false, length) case VarcharEncoder(length) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index 82b3cdc508bf9..06267bca02189 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -22,11 +22,11 @@ import scala.language.existentials import org.apache.spark.sql.catalyst.{expressions => exprs} import org.apache.spark.sql.catalyst.DeserializerBuildHelper.expressionWithNullSafety import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders, AgnosticExpressionPathEncoder, Codec, JavaSerializationCodec, KryoSerializationCodec} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLeafEncoder, BoxedLongEncoder, BoxedShortEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, OptionEncoder, PrimitiveLeafEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, VarcharEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLeafEncoder, BoxedLongEncoder, BoxedShortEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, GeographyEncoder, GeometryEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, OptionEncoder, PrimitiveLeafEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, VarcharEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{externalDataTypeFor, isNativeEncoder, lenientExternalDataTypeFor} import org.apache.spark.sql.catalyst.expressions.{BoundReference, CheckOverflow, CreateNamedStruct, Expression, IsNull, KnownNotNull, Literal, UnsafeArrayData} import org.apache.spark.sql.catalyst.expressions.objects._ -import org.apache.spark.sql.catalyst.util.{ArrayData, CharVarcharCodegenUtils, DateTimeUtils, GenericArrayData, IntervalUtils} +import org.apache.spark.sql.catalyst.util.{ArrayData, CharVarcharCodegenUtils, DateTimeUtils, GenericArrayData, IntervalUtils, STUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -63,6 +63,24 @@ object SerializerBuildHelper { Invoke(inputObject, "doubleValue", DoubleType) } + def createSerializerForGeographyType(inputObject: Expression, gt: GeographyType): Expression = { + StaticInvoke( + classOf[STUtils], + gt, + "serializeGeogFromWKB", + inputObject :: Literal.fromObject(gt) :: Nil, + returnNullable = false) + } + + def createSerializerForGeometryType(inputObject: Expression, gt: GeometryType): Expression = { + StaticInvoke( + classOf[STUtils], + gt, + "serializeGeomFromWKB", + inputObject :: Literal.fromObject(gt) :: Nil, + returnNullable = false) + } + def createSerializerForChar(inputObject: Expression, length: Int): Expression = { StaticInvoke( classOf[CharVarcharCodegenUtils], @@ -326,6 +344,8 @@ object SerializerBuildHelper { case BoxedDoubleEncoder => createSerializerForDouble(input) case JavaEnumEncoder(_) => createSerializerForJavaEnum(input) case ScalaEnumEncoder(_, _) => createSerializerForScalaEnum(input) + case g: GeographyEncoder => createSerializerForGeographyType(input, g.dt) + case g: GeometryEncoder => createSerializerForGeometryType(input, g.dt) case CharEncoder(length) => createSerializerForChar(input, length) case VarcharEncoder(length) => createSerializerForVarchar(input, length) case StringEncoder => createSerializerForString(input) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala new file mode 100644 index 0000000000000..eeb1ba5ea9e25 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.collection.immutable.Seq + +import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +class GeographyDataFrameSuite extends QueryTest with SharedSparkSession { + + val point1 = "010100000000000000000031400000000000001C40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val point2 = "010100000000000000000035400000000000001E40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + + test("decode geography value: SRID schema does not match input SRID data schema") { + val rdd = sparkContext.parallelize(Seq(Row(Geography.fromWKB(point1, 0)))) + val schema = StructType(Seq(StructField("col1", GeographyType(4326), nullable = false))) + checkError( + // We look for cause, as all exception encoder errors are wrapped in + // EXPRESSION_ENCODING_FAILED. + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(rdd, schema).collect() + }.getCause.asInstanceOf[SparkRuntimeException], + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326") + ) + + val javaRDD = sparkContext.parallelize(Seq(Row(Geography.fromWKB(point1, 0)))).toJavaRDD() + checkError( + // We look for cause, as all exception encoder errors are wrapped in + // EXPRESSION_ENCODING_FAILED. + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(javaRDD, schema).collect() + }.getCause.asInstanceOf[SparkRuntimeException], + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326") + ) + + // For some reason this API does not use expression encoders, + // but CatalystTypeConverter, so we are not looking at cause. + val javaList = java.util.Arrays.asList(Row(Geography.fromWKB(point1, 0))) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(javaList, schema).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326") + ) + + val geography1 = Geography.fromWKB(point1, 0) + val rdd2 = sparkContext.parallelize(Seq((geography1, 1))) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(rdd2).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326") + ) + + // For some reason this API does not use expression encoders, + // but CatalystTypeConverter, so we are not looking at cause. + val seq = Seq((geography1, 1)) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(seq).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326") + ) + + import testImplicits._ + checkError( + exception = intercept[SparkRuntimeException] { + Seq(geography1).toDF().collect() + }.getCause.asInstanceOf[SparkRuntimeException], + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326") + ) + } + + test("decode geography value: mixed SRID schema is provided") { + val rdd = sparkContext.parallelize( + Seq(Row(Geography.fromWKB(point1, 4326)), Row(Geography.fromWKB(point2, 4326)))) + val schema = StructType(Seq(StructField("col1", GeographyType("ANY"), nullable = false))) + val expectedResult = Seq( + Row(Geography.fromWKB(point1, 4326)), Row(Geography.fromWKB(point2, 4326))) + + val resultDF = spark.createDataFrame(rdd, schema) + checkAnswer(resultDF, expectedResult) + + val javaRDD = sparkContext.parallelize( + Seq(Row(Geography.fromWKB(point1, 4326)), Row(Geography.fromWKB(point2, 4326)))).toJavaRDD() + val resultJavaDF = spark.createDataFrame(javaRDD, schema) + checkAnswer(resultJavaDF, expectedResult) + + val javaList = java.util.Arrays.asList( + Row(Geography.fromWKB(point1, 4326)), Row(Geography.fromWKB(point2, 4326))) + val resultJavaListDF = spark.createDataFrame(javaList, schema) + checkAnswer(resultJavaListDF, expectedResult) + + // Test that unsupported SRID with mixed schema will throw an error. + val rdd2 = sparkContext.parallelize( + Seq(Row(Geography.fromWKB(point1, 0)), Row(Geography.fromWKB(point2, 4326)))) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(rdd2, schema).collect() + }.getCause.asInstanceOf[SparkIllegalArgumentException], + condition = "ST_INVALID_SRID_VALUE", + parameters = Map("srid" -> "0") + ) + } + + test("createDataFrame APIs with Geography.fromWKB") { + // 1. Test createDataFrame with RDD of Geography objects + val geography1 = Geography.fromWKB(point1, 4326) + val geography2 = Geography.fromWKB(point2) + val rdd = sparkContext.parallelize(Seq((geography1, 1), (geography2, 2), (null, 3))) + val dfFromRDD = spark.createDataFrame(rdd) + checkAnswer(dfFromRDD, Seq(Row(geography1, 1), Row(geography2, 2), Row(null, 3))) + + // 2. Test createDataFrame with Seq of Geography objects + val seq = Seq((geography1, 1), (geography2, 2), (null, 3)) + val dfFromSeq = spark.createDataFrame(seq) + checkAnswer(dfFromSeq, Seq(Row(geography1, 1), Row(geography2, 2), Row(null, 3))) + + // 3. Test createDataFrame with RDD of Rows and StructType schema + val geography3 = Geography.fromWKB(point1, 4326) + val geography4 = Geography.fromWKB(point2, 4326) + val rowRDD = sparkContext.parallelize(Seq(Row(geography3), Row(geography4), Row(null))) + val schema = StructType(Seq( + StructField("geography", GeographyType(4326), nullable = true) + )) + val dfFromRowRDD = spark.createDataFrame(rowRDD, schema) + checkAnswer(dfFromRowRDD, Seq(Row(geography3), Row(geography4), Row(null))) + + // 4. Test createDataFrame with JavaRDD of Rows and StructType schema + val javaRDD = sparkContext.parallelize(Seq(Row(geography3), Row(geography4), Row(null))) + .toJavaRDD() + val dfFromJavaRDD = spark.createDataFrame(javaRDD, schema) + checkAnswer(dfFromJavaRDD, Seq(Row(geography3), Row(geography4), Row(null))) + + // 5. Test createDataFrame with Java List of Rows and StructType schema + val javaList = java.util.Arrays.asList(Row(geography3), Row(geography4), Row(null)) + val dfFromJavaList = spark.createDataFrame(javaList, schema) + checkAnswer(dfFromJavaList, Seq(Row(geography3), Row(geography4), Row(null))) + + // 6. Implicit conversion from Seq to DF + import testImplicits._ + val implicitDf = Seq(geography1, geography2, null).toDF() + checkAnswer(implicitDf, Seq(Row(geography1), Row(geography2), Row(null))) + } + + test("encode geography type") { + // A test WKB value corresponding to: POINT (17 7). + val pointString: String = "010100000000000000000031400000000000001C40" + val pointBytes: Array[Byte] = pointString + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val df = spark.sql(s"SELECT ST_GeogFromWKB(X'$pointString')") + val expectedGeog = Geography.fromWKB(pointBytes, 4326) + checkAnswer(df, Seq(Row(expectedGeog))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala new file mode 100644 index 0000000000000..bcc3cee7ebe39 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.collection.immutable.Seq + +import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +class GeometryDataFrameSuite extends QueryTest with SharedSparkSession { + + val point1 = "010100000000000000000031400000000000001C40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val point2 = "010100000000000000000035400000000000001E40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + + test("decode geometry value: SRID schema does not match input SRID data schema") { + val rdd = sparkContext.parallelize(Seq(Row(Geometry.fromWKB(point1, 0)))) + val schema = StructType(Seq(StructField("col1", GeometryType(3857), nullable = false))) + checkError( + // We look for cause, as all exception encoder errors are wrapped in + // EXPRESSION_ENCODING_FAILED. + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(rdd, schema).collect() + }.getCause.asInstanceOf[SparkRuntimeException], + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "0", "typeSrid" -> "3857") + ) + + val schema2 = StructType(Seq(StructField("col1", GeometryType(0), nullable = false))) + val javaRDD = sparkContext.parallelize(Seq(Row(Geometry.fromWKB(point1, 4326)))).toJavaRDD() + checkError( + // We look for cause, as all exception encoder errors are wrapped in + // EXPRESSION_ENCODING_FAILED. + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(javaRDD, schema2).collect() + }.getCause.asInstanceOf[SparkRuntimeException], + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "0") + ) + + // For some reason this API does not use expression encoders, + // but CatalystTypeConverter, so we are not looking at cause. + val javaList = java.util.Arrays.asList(Row(Geometry.fromWKB(point1, 4326))) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(javaList, schema).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "3857") + ) + + val geometry1 = Geometry.fromWKB(point1, 4326) + val rdd2 = sparkContext.parallelize(Seq((geometry1, 1))) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(rdd2).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "0") + ) + + // For some reason this API does not use expression encoders, + // but CatalystTypeConverter, so we are not looking at cause. + val seq = Seq((geometry1, 1)) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(seq).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "0") + ) + + import testImplicits._ + checkError( + exception = intercept[SparkRuntimeException] { + Seq(geometry1).toDF().collect() + }.getCause.asInstanceOf[SparkRuntimeException], + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "0") + ) + } + + test("decode geometry value: mixed SRID schema is provided") { + val rdd = sparkContext.parallelize( + Seq(Row(Geometry.fromWKB(point1, 0)), Row(Geometry.fromWKB(point2, 4326)))) + val schema = StructType(Seq(StructField("col1", GeometryType("ANY"), nullable = false))) + val expectedResult = Seq( + Row(Geometry.fromWKB(point1, 0)), Row(Geometry.fromWKB(point2, 4326))) + + val resultDF = spark.createDataFrame(rdd, schema) + checkAnswer(resultDF, expectedResult) + + val javaRDD = sparkContext.parallelize( + Seq(Row(Geometry.fromWKB(point1, 0)), Row(Geometry.fromWKB(point2, 4326)))).toJavaRDD() + val resultJavaDF = spark.createDataFrame(javaRDD, schema) + checkAnswer(resultJavaDF, expectedResult) + + val javaList = java.util.Arrays.asList( + Row(Geometry.fromWKB(point1, 0)), Row(Geometry.fromWKB(point2, 4326))) + val resultJavaListDF = spark.createDataFrame(javaList, schema) + checkAnswer(resultJavaListDF, expectedResult) + + // Test that unsupported SRID with mixed schema will throw an error. + val rdd2 = sparkContext.parallelize( + Seq(Row(Geometry.fromWKB(point1, 1)), Row(Geometry.fromWKB(point2, 4326)))) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(rdd2, schema).collect() + }.getCause.asInstanceOf[SparkIllegalArgumentException], + condition = "ST_INVALID_SRID_VALUE", + parameters = Map("srid" -> "1") + ) + } + + test("createDataFrame APIs with Geometry.fromWKB") { + // 1. Test createDataFrame with RDD of Geometry objects + val geometry1 = Geometry.fromWKB(point1, 0) + val geometry2 = Geometry.fromWKB(point2, 0) + val rdd = sparkContext.parallelize(Seq((geometry1, 1), (geometry2, 2), (null, 3))) + val dfFromRDD = spark.createDataFrame(rdd) + checkAnswer(dfFromRDD, Seq(Row(geometry1, 1), Row(geometry2, 2), Row(null, 3))) + + // 2. Test createDataFrame with Seq of Geometry objects + val seq = Seq((geometry1, 1), (geometry2, 2), (null, 3)) + val dfFromSeq = spark.createDataFrame(seq) + checkAnswer(dfFromSeq, Seq(Row(geometry1, 1), Row(geometry2, 2), Row(null, 3))) + + // 3. Test createDataFrame with RDD of Rows and StructType schema + val geometry3 = Geometry.fromWKB(point1, 4326) + val geometry4 = Geometry.fromWKB(point2, 4326) + val rowRDD = sparkContext.parallelize(Seq(Row(geometry3), Row(geometry4), Row(null))) + val schema = StructType(Seq( + StructField("geometry", GeometryType(4326), nullable = true) + )) + val dfFromRowRDD = spark.createDataFrame(rowRDD, schema) + checkAnswer(dfFromRowRDD, Seq(Row(geometry3), Row(geometry4), Row(null))) + + // 4. Test createDataFrame with JavaRDD of Rows and StructType schema + val javaRDD = sparkContext.parallelize(Seq(Row(geometry3), Row(geometry4), Row(null))) + .toJavaRDD() + val dfFromJavaRDD = spark.createDataFrame(javaRDD, schema) + checkAnswer(dfFromJavaRDD, Seq(Row(geometry3), Row(geometry4), Row(null))) + + // 5. Test createDataFrame with Java List of Rows and StructType schema + val javaList = java.util.Arrays.asList(Row(geometry3), Row(geometry4), Row(null)) + val dfFromJavaList = spark.createDataFrame(javaList, schema) + checkAnswer(dfFromJavaList, Seq(Row(geometry3), Row(geometry4), Row(null))) + + // 6. Implicit conversion from Seq to DF + import testImplicits._ + val implicitDf = Seq(geometry1, geometry2, null).toDF() + checkAnswer(implicitDf, Seq(Row(geometry1), Row(geometry2), Row(null))) + } + + test("encode geometry type") { + // A test WKB value corresponding to: POINT (17 7). + val pointString: String = "010100000000000000000031400000000000001C40" + val pointBytes: Array[Byte] = pointString + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val df = spark.sql(s"SELECT ST_GeomFromWKB(X'$pointString')") + val expectedGeom = Geometry.fromWKB(pointBytes, 0) + checkAnswer(df, Seq(Row(expectedGeom))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 5de4170a1c112..eb36b68cd6171 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -136,4 +136,15 @@ class RowSuite extends SparkFunSuite with SharedSparkSession { parameters = Map("index" -> position.toString) ) } + + test("Geospatial row API - Geography and Geometry") { + // A test WKB value corresponding to: POINT (17 7). + val point = "010100000000000000000031400000000000001C40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + + val row = Row(Geometry.fromWKB(point), Geography.fromWKB(point)) + + assert(row.getGeometry(0).getBytes() == point) + assert(row.getGeography(1).getBytes() == point) + } } From ff2847e237989e5c9e740c5154f05da012b7919c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B8rn=20J=C3=B8rgensen?= Date: Mon, 3 Nov 2025 10:56:21 -0800 Subject: [PATCH 019/400] [SPARK-54131][PYTHON][TESTS] Update `Pandas` version 2.3.3 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Update pandas to 2.3.3 ### Why are the changes needed? New version with some bug fixes and support for python 3.14 _Pandas 2.3.3 is the first version of pandas that is generally compatible with the upcoming Python 3.14, and both wheels for free-threaded and normal Python 3.14 will be uploaded for this release._ [Release notes](https://pandas.pydata.org/pandas-docs/version/2.3/whatsnew/index.html#release) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass CI/CD tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52828 from bjornjorgensen/pandas-2_3_3. Authored-by: Bjørn Jørgensen Signed-off-by: Dongjoon Hyun (cherry picked from commit fc3a590d9847251a34545ecbc2c8b0c4170370d4) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_python_connect.yml | 2 +- .github/workflows/pages.yml | 2 +- .github/workflows/python_hosted_runner_test.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 2 +- dev/infra/Dockerfile | 6 +++--- dev/spark-test-image/docs/Dockerfile | 2 +- dev/spark-test-image/pypy-310/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311-classic-only/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- dev/spark-test-image/python-313-nogil/Dockerfile | 4 ++-- dev/spark-test-image/python-313/Dockerfile | 2 +- python/pyspark/pandas/supported_api_gen.py | 2 +- 14 files changed, 17 insertions(+), 17 deletions(-) diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index 5edb54de82b6d..cec37af22dd7b 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -72,7 +72,7 @@ jobs: python packaging/client/setup.py sdist cd dist pip install pyspark*client-*.tar.gz - pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.2' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting + pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting - name: List Python packages run: python -m pip list - name: Run tests (local) diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index 86ef00220b373..e800b40106ee3 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -61,7 +61,7 @@ jobs: - name: Install Python dependencies run: | pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ - ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.2' 'plotly>=4.8' 'docutils<0.18.0' \ + ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index 9a6afc095063c..77e85222c29dd 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -147,7 +147,7 @@ jobs: run: | python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' - python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.2' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ + python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge - name: List Python packages diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index e53ac6b439c87..86be7e0a82295 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -92,7 +92,7 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ # See more in SPARK-39735 ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" -ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index afab64a67b5e5..1aa03735ce926 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -92,10 +92,10 @@ RUN mkdir -p /usr/local/pypy/pypy3.10 && \ ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3.10 && \ ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 -RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.0' scipy coverage matplotlib lxml +RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.3' scipy coverage matplotlib lxml -ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.0 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" @@ -149,7 +149,7 @@ RUN apt-get update && apt-get install -y \ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 # TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.0 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ +RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ python3.13 -m pip cache purge # Remove unused installation packages to free up disk space diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 4c1e68f724479..1c17ae122d638 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -89,7 +89,7 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 # See 'ipython_genutils' in SPARK-38517 # See 'docutils<0.18.0' in SPARK-39421 RUN python3.11 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ - ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.2' 'plotly>=4.8' 'docutils<0.18.0' \ + ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ diff --git a/dev/spark-test-image/pypy-310/Dockerfile b/dev/spark-test-image/pypy-310/Dockerfile index 6f0b938bd1990..c8672fc0ec068 100644 --- a/dev/spark-test-image/pypy-310/Dockerfile +++ b/dev/spark-test-image/pypy-310/Dockerfile @@ -69,4 +69,4 @@ RUN mkdir -p /usr/local/pypy/pypy3.10 && \ ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3.10 && \ ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 -RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.2' scipy coverage matplotlib lxml +RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.3' scipy coverage matplotlib lxml diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index ef59237afee24..ce2ca23d18a64 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-311-classic-only/Dockerfile b/dev/spark-test-image/python-311-classic-only/Dockerfile index 72f7ebec686e8..c2d48c5ce877f 100644 --- a/dev/spark-test-image/python-311-classic-only/Dockerfile +++ b/dev/spark-test-image/python-311-classic-only/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 pandas==2.3.2 plotly<6.0.0 matplotlib openpyxl memory-profiler>=0.61.0 mlflow>=2.8.1 scipy scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 pandas==2.3.3 plotly<6.0.0 matplotlib openpyxl memory-profiler>=0.61.0 mlflow>=2.8.1 scipy scikit-learn>=1.3.2" ARG TEST_PIP_PKGS="coverage unittest-xml-reporting" # Install Python 3.11 packages diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 25fd065753bd3..00fb7be788fd0 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 82016bbec860f..79cab824a5b21 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index 7f608caab1933..031eb8772b59b 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" @@ -77,5 +77,5 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13t # TODO: Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS when it supports Python 3.13 free threaded # TODO: Add lxml, grpcio, grpcio-status back when they support Python 3.13 free threaded RUN python3.13t -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13t -m pip install numpy>=2.1 pyarrow>=19.0.0 six==1.16.0 pandas==2.3.2 scipy coverage matplotlib openpyxl jinja2 && \ +RUN python3.13t -m pip install numpy>=2.1 pyarrow>=19.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl jinja2 && \ python3.13t -m pip cache purge diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 9fd53d233ac07..abd5a7e01093d 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" diff --git a/python/pyspark/pandas/supported_api_gen.py b/python/pyspark/pandas/supported_api_gen.py index 595c11c559d05..3f7efa7784ab7 100644 --- a/python/pyspark/pandas/supported_api_gen.py +++ b/python/pyspark/pandas/supported_api_gen.py @@ -38,7 +38,7 @@ MAX_MISSING_PARAMS_SIZE = 5 COMMON_PARAMETER_SET = {"kwargs", "args", "cls"} MODULE_GROUP_MATCH = [(pd, ps), (pdw, psw), (pdg, psg)] -PANDAS_LATEST_VERSION = "2.3.2" +PANDAS_LATEST_VERSION = "2.3.3" RST_HEADER = """ ===================== From 45f0d90332bdabe9b65544c187b1a91e0381006a Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Mon, 3 Nov 2025 12:04:01 -0800 Subject: [PATCH 020/400] [SPARK-54151][GEO][PYTHON] Introduce the framework for adding ST functions in PySpark ### What changes were proposed in this pull request? This PR adds rudimentary WKB read/write ST geospatial functions in PySpark API, and registers the new Python functions in both PySpark and PySpark Connect. Note that a similar framework was already implemented in Catalyst, as part of: https://github.com/apache/spark/pull/52784. ### Why are the changes needed? Establish a minimal ST function framework in PySpark API, setting the foundations for expanding geospatial function support in the near future. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces 3 new Python functions: `st_asbinary`, `st_geogfromwkb`, `st_geomfromwkb`. ### How was this patch tested? Added new PySpark unit test suites: - `test_functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52849 from uros-db/geo-python-functions. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit 24110b6f6b1b05176bdb5bde80c8f2144b87c9cc) Signed-off-by: Wenchen Fan --- .../reference/pyspark.sql/functions.rst | 10 +++ .../pyspark/sql/connect/functions/builtin.py | 24 ++++++ python/pyspark/sql/functions/__init__.py | 5 ++ python/pyspark/sql/functions/builtin.py | 73 +++++++++++++++++++ python/pyspark/sql/tests/test_functions.py | 17 +++++ 5 files changed, 129 insertions(+) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index e4175707aecd7..6576c7245e311 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -652,6 +652,16 @@ Misc Functions version +Geospatial ST Functions +----------------------- +.. autosummary:: + :toctree: api/ + + st_asbinary + st_geogfromwkb + st_geomfromwkb + + UDF, UDTF and UDT ----------------- .. autosummary:: diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 1198596fbb5db..2c58ed946a82c 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -4783,6 +4783,30 @@ def bitmap_and_agg(col: "ColumnOrName") -> Column: bitmap_and_agg.__doc__ = pysparkfuncs.bitmap_and_agg.__doc__ +# Geospatial ST Functions + + +def st_asbinary(geo: "ColumnOrName") -> Column: + return _invoke_function_over_columns("st_asbinary", geo) + + +st_asbinary.__doc__ = pysparkfuncs.st_asbinary.__doc__ + + +def st_geogfromwkb(wkb: "ColumnOrName") -> Column: + return _invoke_function_over_columns("st_geogfromwkb", wkb) + + +st_geogfromwkb.__doc__ = pysparkfuncs.st_geogfromwkb.__doc__ + + +def st_geomfromwkb(wkb: "ColumnOrName") -> Column: + return _invoke_function_over_columns("st_geomfromwkb", wkb) + + +st_geomfromwkb.__doc__ = pysparkfuncs.st_geomfromwkb.__doc__ + + # Call Functions diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index e1b320c98f7fe..df9594f18c96d 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -522,6 +522,11 @@ "UserDefinedFunction", "UserDefinedTableFunction", "arrow_udf", + # Geospatial ST Functions + "st_asbinary", + "st_geogfromwkb", + "st_geomfromwkb", + # Call Functions "call_udf", "pandas_udf", "udf", diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 1ac3ac23e888e..ade6723485e26 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -25901,6 +25901,79 @@ def bucket(numBuckets: Union[Column, int], col: "ColumnOrName") -> Column: return partitioning.bucket(numBuckets, col) +# Geospatial ST Functions + + +@_try_remote_functions +def st_asbinary(geo: "ColumnOrName") -> Column: + """Returns the input GEOGRAPHY or GEOMETRY value in WKB format. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + geo : :class:`~pyspark.sql.Column` or str + A geospatial value, either a GEOGRAPHY or a GEOMETRY. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geogfromwkb('wkb'))).alias('result')).collect() + [Row(result='0101000000000000000000F03F0000000000000040')] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geomfromwkb('wkb'))).alias('result')).collect() + [Row(result='0101000000000000000000F03F0000000000000040')] + """ + return _invoke_function_over_columns("st_asbinary", geo) + + +@_try_remote_functions +def st_geogfromwkb(wkb: "ColumnOrName") -> Column: + """Parses the input WKB description and returns the corresponding GEOGRAPHY value. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + wkb : :class:`~pyspark.sql.Column` or str + A BINARY value in WKB format, representing a GEOGRAPHY value. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geogfromwkb('wkb'))).alias('result')).collect() + [Row(result='0101000000000000000000F03F0000000000000040')] + """ + return _invoke_function_over_columns("st_geogfromwkb", wkb) + + +@_try_remote_functions +def st_geomfromwkb(wkb: "ColumnOrName") -> Column: + """Parses the input WKB description and returns the corresponding GEOMETRY value. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + wkb : :class:`~pyspark.sql.Column` or str + A BINARY value in WKB format, representing a GEOMETRY value. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geomfromwkb('wkb'))).alias('result')).collect() + [Row(result='0101000000000000000000F03F0000000000000040')] + """ + return _invoke_function_over_columns("st_geomfromwkb", wkb) + + +# Call Functions + + @_try_remote_functions def call_udf(udfName: str, *cols: "ColumnOrName") -> Column: """ diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 18f824c463c93..6dc0770d3df41 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -2805,6 +2805,23 @@ def test_string_validation(self): result_try_validate_utf8 = df.select(F.try_validate_utf8(df.a).alias("r")) assertDataFrameEqual([Row(r="abc")], result_try_validate_utf8) + # Geospatial ST Functions + + def test_st_asbinary(self): + df = self.spark.createDataFrame( + [(bytes.fromhex("0101000000000000000000F03F0000000000000040"),)], + ["wkb"], + ) + results = df.select( + F.hex(F.st_asbinary(F.st_geogfromwkb("wkb"))), + F.hex(F.st_asbinary(F.st_geomfromwkb("wkb"))), + ).collect() + expected = Row( + "0101000000000000000000F03F0000000000000040", + "0101000000000000000000F03F0000000000000040", + ) + self.assertEqual(results, [expected]) + class FunctionsTests(ReusedSQLTestCase, FunctionsTestsMixin): pass From 074fb66edcda838dd853b44d64547feb1c61714c Mon Sep 17 00:00:00 2001 From: Chris Boumalhab Date: Mon, 3 Nov 2025 12:08:39 -0800 Subject: [PATCH 021/400] [SPARK-54138][SQL] Enforce constant configuration parameter for Theta and HLL ### What changes were proposed in this pull request? This PR enforces that the sketch configuration parameter (lgConfigK/lgNomEntries) in both HllSketchAgg and ThetaSketchAgg must be a constant value. If the parameter expression (right) is not foldable, a QueryExecutionErrors.*MustBeConstantError(prettyName) is thrown. This change ensures that the aggregation configuration is validated at analysis time rather than runtime, preventing inconsistent or invalid sketch behavior. ### Why are the changes needed? The configuration parameter determines the accuracy and memory footprint of the sketch. Allowing it to vary dynamically at runtime could lead to nondeterministic aggregation results and incorrect computations. By enforcing it as a constant expression, we ensure deterministic behavior, predictable memory use, and alignment with the expected semantics of sketch-based aggregations. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added tests in SQLQueryTestSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #52836 from cboumalh/hll-theta-check-foldable-param. Authored-by: Chris Boumalhab Signed-off-by: Daniel Tenedorio (cherry picked from commit be6ab9cfafbe3b239c2bde4c48d434cfee03f876) Signed-off-by: Daniel Tenedorio --- .../resources/error/error-conditions.json | 12 ++++ .../aggregate/datasketchesAggregates.scala | 3 + .../aggregate/thetasketchesAggregates.scala | 6 ++ .../sql/errors/QueryExecutionErrors.scala | 12 ++++ .../sql-tests/analyzer-results/hll.sql.out | 43 +++++++++++++ .../analyzer-results/thetasketch.sql.out | 43 +++++++++++++ .../test/resources/sql-tests/inputs/hll.sql | 9 +++ .../sql-tests/inputs/thetasketch.sql | 12 ++++ .../resources/sql-tests/results/hll.sql.out | 62 +++++++++++++++++++ .../sql-tests/results/thetasketch.sql.out | 62 +++++++++++++++++++ 10 files changed, 264 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 9d95d74cc21a1..a34ceb9f11454 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1972,6 +1972,12 @@ ], "sqlState" : "22546" }, + "HLL_K_MUST_BE_CONSTANT" : { + "message" : [ + "Invalid call to ; the `K` value must be a constant value, but got a non-constant expression." + ], + "sqlState" : "42K0E" + }, "HLL_UNION_DIFFERENT_LG_K" : { "message" : [ "Sketches have different `lgConfigK` values: and . Set the `allowDifferentLgConfigK` parameter to true to call with different `lgConfigK` values." @@ -5757,6 +5763,12 @@ ], "sqlState" : "22546" }, + "THETA_LG_NOM_ENTRIES_MUST_BE_CONSTANT" : { + "message" : [ + "Invalid call to ; the `lgNomEntries` value must be a constant value, but got a non-constant expression." + ], + "sqlState" : "42K0E" + }, "TRAILING_COMMA_IN_SELECT" : { "message" : [ "Trailing comma detected in SELECT clause. Remove the trailing comma before the FROM clause." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala index cfcb53769e253..8ae9b8fddde71 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala @@ -66,6 +66,9 @@ case class HllSketchAgg( // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation. lazy val lgConfigK: Int = { + if (!right.foldable) { + throw QueryExecutionErrors.hllKMustBeConstantError(prettyName) + } val lgConfigK = right.eval().asInstanceOf[Int] HllSketchAgg.checkLgK(lgConfigK) lgConfigK diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala index 7e55c006782cf..f841632b14622 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala @@ -94,6 +94,9 @@ case class ThetaSketchAgg( // ThetaSketch config - mark as lazy so that they're not evaluated during tree transformation. lazy val lgNomEntries: Int = { + if (!right.foldable) { + throw QueryExecutionErrors.thetaLgNomEntriesMustBeConstantError(prettyName) + } val lgNomEntriesInput = right.eval().asInstanceOf[Int] ThetaSketchUtils.checkLgNomLongs(lgNomEntriesInput, prettyName) lgNomEntriesInput @@ -332,6 +335,9 @@ case class ThetaUnionAgg( // ThetaSketch config - mark as lazy so that they're not evaluated during tree transformation. lazy val lgNomEntries: Int = { + if (!right.foldable) { + throw QueryExecutionErrors.thetaLgNomEntriesMustBeConstantError(prettyName) + } val lgNomEntriesInput = right.eval().asInstanceOf[Int] ThetaSketchUtils.checkLgNomLongs(lgNomEntriesInput, prettyName) lgNomEntriesInput diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 5f5e1da47184c..55dcea57ff322 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -2804,6 +2804,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "value" -> toSQLValue(value, IntegerType))) } + def hllKMustBeConstantError(function: String): Throwable = { + new SparkRuntimeException( + errorClass = "HLL_K_MUST_BE_CONSTANT", + messageParameters = Map("function" -> toSQLId(function))) + } + def hllInvalidInputSketchBuffer(function: String): Throwable = { new SparkRuntimeException( errorClass = "HLL_INVALID_INPUT_SKETCH_BUFFER", @@ -3169,4 +3175,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "max" -> toSQLValue(max, IntegerType), "value" -> toSQLValue(value, IntegerType))) } + + def thetaLgNomEntriesMustBeConstantError(function: String): Throwable = { + new SparkRuntimeException( + errorClass = "THETA_LG_NOM_ENTRIES_MUST_BE_CONSTANT", + messageParameters = Map("function" -> toSQLId(function))) + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out index 167c8f930d25d..291f071ef06c2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out @@ -233,6 +233,49 @@ Aggregate [hll_sketch_agg(col#x, 40, 0, 0) AS hll_sketch_agg(col, 40)#x] +- LocalRelation [col#x] +-- !query +SELECT hll_sketch_agg(col, CAST(NULL AS INT)) AS k_is_null +FROM VALUES (15), (16), (17) tab(col) +-- !query analysis +Aggregate [hll_sketch_agg(col#x, cast(null as int), 0, 0) AS k_is_null#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT hll_sketch_agg(col, CAST(col AS INT)) AS k_non_constant +FROM VALUES (15), (16), (17) tab(col) +-- !query analysis +Aggregate [hll_sketch_agg(col#x, cast(col#x as int), 0, 0) AS k_non_constant#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT hll_sketch_agg(col, '15') +FROM VALUES (50), (60), (60) tab(col) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"15\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"INT\"", + "sqlExpr" : "\"hll_sketch_agg(col, 15)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "hll_sketch_agg(col, '15')" + } ] +} + + -- !query SELECT hll_union( hll_sketch_agg(col1, 12), diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/thetasketch.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/thetasketch.sql.out index 323084223d4bc..84fb8086151d1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/thetasketch.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/thetasketch.sql.out @@ -1068,6 +1068,49 @@ Aggregate [theta_union_agg(sketch#x, 27, 0, 0) AS theta_union_agg(sketch, 27)#x] +- LocalRelation [col#x] +-- !query +SELECT theta_sketch_agg(col, CAST(NULL AS INT)) AS lg_nom_entries_is_null +FROM VALUES (15), (16), (17) tab(col) +-- !query analysis +Aggregate [theta_sketch_agg(col#x, cast(null as int), 0, 0) AS lg_nom_entries_is_null#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT theta_sketch_agg(col, CAST(col AS INT)) AS lg_nom_entries_non_constant +FROM VALUES (15), (16), (17) tab(col) +-- !query analysis +Aggregate [theta_sketch_agg(col#x, cast(col#x as int), 0, 0) AS lg_nom_entries_non_constant#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT theta_sketch_agg(col, '15') +FROM VALUES (50), (60), (60) tab(col) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"15\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"INT\"", + "sqlExpr" : "\"theta_sketch_agg(col, 15)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "theta_sketch_agg(col, '15')" + } ] +} + + -- !query SELECT theta_union(1, 2) FROM VALUES diff --git a/sql/core/src/test/resources/sql-tests/inputs/hll.sql b/sql/core/src/test/resources/sql-tests/inputs/hll.sql index fbd82b936b776..35128da97fd61 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/hll.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/hll.sql @@ -69,6 +69,15 @@ FROM VALUES (50), (60), (60) tab(col); SELECT hll_sketch_agg(col, 40) FROM VALUES (50), (60), (60) tab(col); +SELECT hll_sketch_agg(col, CAST(NULL AS INT)) AS k_is_null +FROM VALUES (15), (16), (17) tab(col); + +SELECT hll_sketch_agg(col, CAST(col AS INT)) AS k_non_constant +FROM VALUES (15), (16), (17) tab(col); + +SELECT hll_sketch_agg(col, '15') +FROM VALUES (50), (60), (60) tab(col); + SELECT hll_union( hll_sketch_agg(col1, 12), hll_sketch_agg(col2, 13)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/thetasketch.sql b/sql/core/src/test/resources/sql-tests/inputs/thetasketch.sql index d270442b50499..4782d2017f2a6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/thetasketch.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/thetasketch.sql @@ -457,6 +457,18 @@ FROM (SELECT theta_sketch_agg(col, 12) as sketch SELECT theta_sketch_agg(col, 20) as sketch FROM VALUES (1) AS tab(col)); +-- lgNomEntries parameter is NULL +SELECT theta_sketch_agg(col, CAST(NULL AS INT)) AS lg_nom_entries_is_null +FROM VALUES (15), (16), (17) tab(col); + +-- lgNomEntries parameter is not foldable (non-constant) +SELECT theta_sketch_agg(col, CAST(col AS INT)) AS lg_nom_entries_non_constant +FROM VALUES (15), (16), (17) tab(col); + +-- lgNomEntries parameter has wrong type (STRING instead of INT) +SELECT theta_sketch_agg(col, '15') +FROM VALUES (50), (60), (60) tab(col); + -- Test theta_union with integers (1, 2) instead of binary sketch data - should fail SELECT theta_union(1, 2) FROM VALUES diff --git a/sql/core/src/test/resources/sql-tests/results/hll.sql.out b/sql/core/src/test/resources/sql-tests/results/hll.sql.out index ecdfcbcc791a3..908221f0e7c40 100644 --- a/sql/core/src/test/resources/sql-tests/results/hll.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/hll.sql.out @@ -256,6 +256,68 @@ org.apache.spark.SparkRuntimeException } +-- !query +SELECT hll_sketch_agg(col, CAST(NULL AS INT)) AS k_is_null +FROM VALUES (15), (16), (17) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_LG_K", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`hll_sketch_agg`", + "max" : "21", + "min" : "4", + "value" : "0" + } +} + + +-- !query +SELECT hll_sketch_agg(col, CAST(col AS INT)) AS k_non_constant +FROM VALUES (15), (16), (17) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_K_MUST_BE_CONSTANT", + "sqlState" : "42K0E", + "messageParameters" : { + "function" : "`hll_sketch_agg`" + } +} + + +-- !query +SELECT hll_sketch_agg(col, '15') +FROM VALUES (50), (60), (60) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"15\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"INT\"", + "sqlExpr" : "\"hll_sketch_agg(col, 15)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "hll_sketch_agg(col, '15')" + } ] +} + + -- !query SELECT hll_union( hll_sketch_agg(col1, 12), diff --git a/sql/core/src/test/resources/sql-tests/results/thetasketch.sql.out b/sql/core/src/test/resources/sql-tests/results/thetasketch.sql.out index 95c6e28a8c426..685a268b434d2 100644 --- a/sql/core/src/test/resources/sql-tests/results/thetasketch.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/thetasketch.sql.out @@ -976,6 +976,68 @@ org.apache.spark.SparkRuntimeException } +-- !query +SELECT theta_sketch_agg(col, CAST(NULL AS INT)) AS lg_nom_entries_is_null +FROM VALUES (15), (16), (17) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "THETA_INVALID_LG_NOM_ENTRIES", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`theta_sketch_agg`", + "max" : "26", + "min" : "4", + "value" : "0" + } +} + + +-- !query +SELECT theta_sketch_agg(col, CAST(col AS INT)) AS lg_nom_entries_non_constant +FROM VALUES (15), (16), (17) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "THETA_LG_NOM_ENTRIES_MUST_BE_CONSTANT", + "sqlState" : "42K0E", + "messageParameters" : { + "function" : "`theta_sketch_agg`" + } +} + + +-- !query +SELECT theta_sketch_agg(col, '15') +FROM VALUES (50), (60), (60) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"15\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"INT\"", + "sqlExpr" : "\"theta_sketch_agg(col, 15)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "theta_sketch_agg(col, '15')" + } ] +} + + -- !query SELECT theta_union(1, 2) FROM VALUES From 927cfcc2d008ce5351e84894a95a2e742982b68f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 3 Nov 2025 13:07:38 -0800 Subject: [PATCH 022/400] [SPARK-54134][SQL] Optimize Arrow memory usage ### What changes were proposed in this pull request? This patch proposes some changes to optimize memory usage on Arrow in Spark. It compress Arrow IPC data when serializing. ### Why are the changes needed? We have encountered OOM when loading data and processing them in PySpark through `toArrow` or `toPandas`. The same data could be loaded by PyArrow directly but fails to load through `toArrow` or `toPandas` into PySpark due to OOM issues. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit tests. Manually test it locally. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Code v2.0.13 Closes #52747 from viirya/release_buffers. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun (cherry picked from commit 02ba89f0299314decc6f7afc3603a7d7e3bdc019) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 1 + pom.xml | 19 +++++ python/pyspark/sql/tests/arrow/test_arrow.py | 75 +++++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 16 ++++ sql/core/pom.xml | 4 + .../sql/execution/arrow/ArrowConverters.scala | 24 +++++- 6 files changed, 137 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 5c4c053293e02..73f36fb161c66 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -15,6 +15,7 @@ antlr4-runtime/4.13.1//antlr4-runtime-4.13.1.jar aopalliance-repackaged/3.0.6//aopalliance-repackaged-3.0.6.jar arpack/3.0.4//arpack-3.0.4.jar arpack_combined_all/0.1//arpack_combined_all-0.1.jar +arrow-compression/18.3.0//arrow-compression-18.3.0.jar arrow-format/18.3.0//arrow-format-18.3.0.jar arrow-memory-core/18.3.0//arrow-memory-core-18.3.0.jar arrow-memory-netty-buffer-patch/18.3.0//arrow-memory-netty-buffer-patch-18.3.0.jar diff --git a/pom.xml b/pom.xml index 812ffa89d4fc2..df8f76c47115b 100644 --- a/pom.xml +++ b/pom.xml @@ -2519,6 +2519,25 @@ + + org.apache.arrow + arrow-compression + ${arrow.version} + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + io.netty + netty-common + + + org.apache.arrow arrow-memory-netty diff --git a/python/pyspark/sql/tests/arrow/test_arrow.py b/python/pyspark/sql/tests/arrow/test_arrow.py index be7dd2febc94a..af08f8c8c1012 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow.py +++ b/python/pyspark/sql/tests/arrow/test_arrow.py @@ -1810,6 +1810,81 @@ def test_createDataFrame_arrow_fixed_size_list(self): df = self.spark.createDataFrame(t) self.assertIsInstance(df.schema["fsl"].dataType, ArrayType) + def test_toPandas_with_compression_codec(self): + # Test toPandas() with different compression codec settings + df = self.spark.createDataFrame(self.data, schema=self.schema) + expected = self.create_pandas_data_frame() + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + pdf = df.toPandas() + assert_frame_equal(expected, pdf) + + def test_toArrow_with_compression_codec(self): + # Test toArrow() with different compression codec settings + import pyarrow.compute as pc + + t_in = self.create_arrow_table() + + # Convert timezone-naive local timestamp column in input table to UTC + # to enable comparison to UTC timestamp column in output table + timezone = self.spark.conf.get("spark.sql.session.timeZone") + t_in = t_in.set_column( + t_in.schema.get_field_index("8_timestamp_t"), + "8_timestamp_t", + pc.assume_timezone(t_in["8_timestamp_t"], timezone), + ) + t_in = t_in.cast( + t_in.schema.set( + t_in.schema.get_field_index("8_timestamp_t"), + pa.field("8_timestamp_t", pa.timestamp("us", tz="UTC")), + ) + ) + + df = self.spark.createDataFrame(self.data, schema=self.schema) + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + t_out = df.toArrow() + self.assertTrue(t_out.equals(t_in)) + + def test_toPandas_with_compression_codec_large_dataset(self): + # Test compression with a larger dataset to verify memory savings + # Create a dataset with repetitive data that compresses well + from pyspark.sql.functions import lit, col + + df = self.spark.range(10000).select( + col("id"), + lit("test_string_value_" * 10).alias("str_col"), + (col("id") % 100).alias("mod_col"), + ) + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + pdf = df.toPandas() + self.assertEqual(len(pdf), 10000) + self.assertEqual(pdf.columns.tolist(), ["id", "str_col", "mod_col"]) + + def test_toArrow_with_compression_codec_large_dataset(self): + # Test compression with a larger dataset for toArrow + from pyspark.sql.functions import lit, col + + df = self.spark.range(10000).select( + col("id"), + lit("test_string_value_" * 10).alias("str_col"), + (col("id") % 100).alias("mod_col"), + ) + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + t = df.toArrow() + self.assertEqual(t.num_rows, 10000) + self.assertEqual(t.column_names, ["id", "str_col", "mod_col"]) + @unittest.skipIf( not have_pandas or not have_pyarrow, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d88cbe326cfbe..b8907629ad372 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3987,6 +3987,20 @@ object SQLConf { "than zero and less than INT_MAX.") .createWithDefaultString("64MB") + val ARROW_EXECUTION_COMPRESSION_CODEC = + buildConf("spark.sql.execution.arrow.compressionCodec") + .doc("Compression codec used to compress Arrow IPC data when transferring data " + + "between JVM and Python processes (e.g., toPandas, toArrow). This can significantly " + + "reduce memory usage and network bandwidth when transferring large datasets. " + + "Supported codecs: 'none' (no compression), 'zstd' (Zstandard), 'lz4' (LZ4). " + + "Note that compression may add CPU overhead but can provide substantial memory savings " + + "especially for datasets with high compression ratios.") + .version("4.1.0") + .stringConf + .transform(_.toLowerCase(java.util.Locale.ROOT)) + .checkValues(Set("none", "zstd", "lz4")) + .createWithDefault("none") + val ARROW_TRANSFORM_WITH_STATE_IN_PYSPARK_MAX_STATE_RECORDS_PER_BATCH = buildConf("spark.sql.execution.arrow.transformWithStateInPySpark.maxStateRecordsPerBatch") .doc("When using TransformWithState in PySpark (both Python Row and Pandas), limit " + @@ -7332,6 +7346,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def arrowMaxBytesPerBatch: Long = getConf(ARROW_EXECUTION_MAX_BYTES_PER_BATCH) + def arrowCompressionCodec: String = getConf(ARROW_EXECUTION_COMPRESSION_CODEC) + def arrowTransformWithStateInPySparkMaxStateRecordsPerBatch: Int = getConf(ARROW_TRANSFORM_WITH_STATE_IN_PYSPARK_MAX_STATE_RECORDS_PER_BATCH) diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 39d8c39954410..cf7494212cd55 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -279,6 +279,10 @@ bcpkix-jdk18on test + + org.apache.arrow + arrow-compression + target/scala-${scala.binary.version}/classes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 3072a12e3d587..7f260bd2efd04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -23,9 +23,11 @@ import java.nio.channels.{Channels, ReadableByteChannel} import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ +import org.apache.arrow.compression.{Lz4CompressionCodec, ZstdCompressionCodec} import org.apache.arrow.flatbuf.MessageHeader import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector._ +import org.apache.arrow.vector.compression.{CompressionCodec, NoCompressionCodec} import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter, ReadChannel, WriteChannel} import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, IpcOption, MessageSerializer} @@ -37,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} @@ -92,8 +95,25 @@ private[sql] object ArrowConverters extends Logging { ArrowUtils.rootAllocator.newChildAllocator( s"to${this.getClass.getSimpleName}", 0, Long.MaxValue) - private val root = VectorSchemaRoot.create(arrowSchema, allocator) - protected val unloader = new VectorUnloader(root) + protected val root = VectorSchemaRoot.create(arrowSchema, allocator) + + // Create compression codec based on config + private val compressionCodecName = SQLConf.get.arrowCompressionCodec + private val codec = compressionCodecName match { + case "none" => NoCompressionCodec.INSTANCE + case "zstd" => + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new ZstdCompressionCodec().getCodecType() + factory.createCodec(codecType) + case "lz4" => + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new Lz4CompressionCodec().getCodecType() + factory.createCodec(codecType) + case other => + throw new IllegalArgumentException( + s"Unsupported Arrow compression codec: $other. Supported values: none, zstd, lz4") + } + protected val unloader = new VectorUnloader(root, true, codec, true) protected val arrowWriter = ArrowWriter.create(root) Option(context).foreach {_.addTaskCompletionListener[Unit] { _ => From d6fbe93aa076da2682f2b28df2c65cbde11b930d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 3 Nov 2025 13:16:44 -0800 Subject: [PATCH 023/400] [SPARK-54155][BUILD][TESTS] Upgrade `mssql-jdbc` to `13.2.1.jre11` ### What changes were proposed in this pull request? This PR aims to upgrade `mssql-jdbc` test dependency to `13.2.1.jre11`. ### Why are the changes needed? To use the latest features (like JDK 23 official support) and bug fixed versions during testing. - https://github.com/microsoft/mssql-jdbc/releases/tag/v13.2.1 - https://github.com/microsoft/mssql-jdbc/pull/2801 - https://github.com/microsoft/mssql-jdbc/releases/tag/v13.2.0 - https://github.com/microsoft/mssql-jdbc/pull/2724 - https://github.com/microsoft/mssql-jdbc/releases/tag/v12.10.2 - https://github.com/microsoft/mssql-jdbc/releases/tag/v12.10.1 - https://github.com/microsoft/mssql-jdbc/releases/tag/v12.10.0 - https://github.com/microsoft/mssql-jdbc/releases/tag/v12.9.0 - https://github.com/microsoft/mssql-jdbc/pull/2515 - https://github.com/microsoft/mssql-jdbc/releases/tag/v12.8.2 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52856 from dongjoon-hyun/SPARK-54155. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 34c3dde17df21fddb3f3624c8c5000008c2b39e6) Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index df8f76c47115b..b1c250a34c723 100644 --- a/pom.xml +++ b/pom.xml @@ -344,7 +344,7 @@ 9.2.0 42.7.7 11.5.9.0 - 12.8.1.jre11 + 13.2.1.jre11 23.6.0.24.10 2.7.1 3.26.1 From 67f2ac5e6ecdf8083b1ef67460869d8f0bff2790 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 3 Nov 2025 13:27:40 -0800 Subject: [PATCH 024/400] [SPARK-54145][PYTHON][CONNECT] Fix column check of nested type in numeric aggregation ### What changes were proposed in this pull request? Fix column check in numeric agg ### Why are the changes needed? query: ```py df = spark.createDataFrame( [ Row(a="a", b=Row(c=1)), Row(a="a", b=Row(c=2)), Row(a="a", b=Row(c=3)), Row(a="b", b=Row(c=4)), Row(a="b", b=Row(c=5)), ] ) df.groupBy("a").max("b.c").show() ``` in classic: ``` +---+-------------+ | a|max(b.c AS c)| +---+-------------+ | a| 3| | b| 5| +---+-------------+ ``` in connect: ``` --------------------------------------------------------------------------- PySparkTypeError Traceback (most recent call last) Cell In[2], line 11 1 df = spark.createDataFrame( 2 [ 3 Row(a="a", b=Row(c=1)), (...) 8 ] 9 ) ---> 11 df.groupBy("a").max("b.c").show() File ~/spark/python/pyspark/sql/connect/group.py:203, in GroupedData.max(self, *cols) 202 def max(self: "GroupedData", *cols: str) -> "DataFrame": --> 203 return self._numeric_agg("max", list(cols)) File ~/spark/python/pyspark/sql/connect/group.py:175, in GroupedData._numeric_agg(self, function, cols) 173 invalid_cols = [c for c in cols if c not in numerical_cols] 174 if len(invalid_cols) > 0: --> 175 raise PySparkTypeError( 176 errorClass="NOT_NUMERIC_COLUMNS", 177 messageParameters={"invalid_columns": str(invalid_cols)}, 178 ) 179 agg_cols = cols 180 else: 181 # if no column is provided, then all numerical columns are selected PySparkTypeError: [NOT_NUMERIC_COLUMNS] Numeric aggregation function can only be applied on numeric columns, got ['b.c']. ``` ### Does this PR introduce _any_ user-facing change? yes, above query works after this fix ### How was this patch tested? added test ### Was this patch authored or co-authored using generative AI tooling? no Closes #52844 from zhengruifeng/numeric_agg_nest_col. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 94e00ca8f5fcdc1d3c430e1b54de8012101f1d8d) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/connect/group.py | 11 ++++---- python/pyspark/sql/connect/types.py | 38 ++++++++++++++++++++++---- python/pyspark/sql/tests/test_group.py | 16 +++++++++++ 3 files changed, 54 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 52d280c2c2646..d540e721f149e 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -158,6 +158,7 @@ def agg(self, *exprs: Union[Column, Dict[str, str]]) -> "DataFrame": def _numeric_agg(self, function: str, cols: Sequence[str]) -> "DataFrame": from pyspark.sql.connect.dataframe import DataFrame + from pyspark.sql.connect.types import verify_numeric_col_name assert isinstance(function, str) and function in ["min", "max", "avg", "sum"] @@ -165,12 +166,8 @@ def _numeric_agg(self, function: str, cols: Sequence[str]) -> "DataFrame": schema = self._df.schema - numerical_cols: List[str] = [ - field.name for field in schema.fields if isinstance(field.dataType, NumericType) - ] - if len(cols) > 0: - invalid_cols = [c for c in cols if c not in numerical_cols] + invalid_cols = [c for c in cols if not verify_numeric_col_name(c, schema)] if len(invalid_cols) > 0: raise PySparkTypeError( errorClass="NOT_NUMERIC_COLUMNS", @@ -179,7 +176,9 @@ def _numeric_agg(self, function: str, cols: Sequence[str]) -> "DataFrame": agg_cols = cols else: # if no column is provided, then all numerical columns are selected - agg_cols = numerical_cols + agg_cols = [ + field.name for field in schema.fields if isinstance(field.dataType, NumericType) + ] return DataFrame( plan.Aggregate( diff --git a/python/pyspark/sql/connect/types.py b/python/pyspark/sql/connect/types.py index 8f9e7c0561cc0..7e8f768610794 100644 --- a/python/pyspark/sql/connect/types.py +++ b/python/pyspark/sql/connect/types.py @@ -48,6 +48,7 @@ BinaryType, BooleanType, NullType, + NumericType, VariantType, UserDefinedType, ) @@ -367,15 +368,42 @@ def verify_col_name(name: str, schema: StructType) -> bool: if parts is None or len(parts) == 0: return False - def _quick_verify(parts: List[str], schema: DataType) -> bool: + def _quick_verify(parts: List[str], dt: DataType) -> bool: if len(parts) == 0: return True _schema: Optional[StructType] = None - if isinstance(schema, StructType): - _schema = schema - elif isinstance(schema, ArrayType) and isinstance(schema.elementType, StructType): - _schema = schema.elementType + if isinstance(dt, StructType): + _schema = dt + elif isinstance(dt, ArrayType) and isinstance(dt.elementType, StructType): + _schema = dt.elementType + else: + return False + + part = parts[0] + for field in _schema: + if field.name == part: + return _quick_verify(parts[1:], field.dataType) + + return False + + return _quick_verify(parts, schema) + + +def verify_numeric_col_name(name: str, schema: StructType) -> bool: + parts = parse_attr_name(name) + if parts is None or len(parts) == 0: + return False + + def _quick_verify(parts: List[str], dt: DataType) -> bool: + if len(parts) == 0 and isinstance(dt, NumericType): + return True + + _schema: Optional[StructType] = None + if isinstance(dt, StructType): + _schema = dt + elif isinstance(dt, ArrayType) and isinstance(dt.elementType, StructType): + _schema = dt.elementType else: return False diff --git a/python/pyspark/sql/tests/test_group.py b/python/pyspark/sql/tests/test_group.py index bbc089b00c133..ac868c34a9133 100644 --- a/python/pyspark/sql/tests/test_group.py +++ b/python/pyspark/sql/tests/test_group.py @@ -126,6 +126,22 @@ def test_group_by_ordinal(self): with self.assertRaises(IndexError): df.groupBy(10).agg(sf.sum("b")) + def test_numeric_agg_with_nest_type(self): + df = self.spark.createDataFrame( + [ + Row(a="a", b=Row(c=1)), + Row(a="a", b=Row(c=2)), + Row(a="a", b=Row(c=3)), + Row(a="b", b=Row(c=4)), + Row(a="b", b=Row(c=5)), + ] + ) + + res = df.groupBy("a").max("b.c").sort("a").collect() + # [Row(a='a', max(b.c AS c)=3), Row(a='b', max(b.c AS c)=5)] + + self.assertEqual([["a", 3], ["b", 5]], [list(r) for r in res]) + @unittest.skipIf(not have_pandas, pandas_requirement_message) # type: ignore @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) # type: ignore def test_order_by_ordinal(self): From 337eb8546a53f80706fcaf609bb4d1d7696f36b9 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 3 Nov 2025 13:49:03 -0800 Subject: [PATCH 025/400] [SPARK-53128][CORE] Include unmanaged memory bytes in the usage log before execution memory OOM ### What changes were proposed in this pull request? We have a log before OOM for off-heap memory allocation. Before the change, the log is: > 25/08/05 16:44:32 INFO TaskMemoryManager: 100 bytes of memory are used for execution and 100 bytes of memory are used for storage After: > 25/08/05 16:44:32 INFO TaskMemoryManager: 100 bytes of memory are used for execution and 100 bytes of memory are used for storage and 500 bytes of memory are used but unmanaged ### Why are the changes needed? Following https://github.com/apache/spark/pull/51708, to allow user to know the reason if the unmanaged memory causes OOM. ### Does this PR introduce _any_ user-facing change? Only changes a log message. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #51848 from zhztheplayer/wip-53128. Authored-by: Hongze Zhang Signed-off-by: Dongjoon Hyun (cherry picked from commit c4ad38117e4be3fed7534e0159c1203a01f058e1) Signed-off-by: Dongjoon Hyun --- .../java/org/apache/spark/memory/TaskMemoryManager.java | 7 +++++-- .../org/apache/spark/memory/UnifiedMemoryManager.scala | 8 ++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index ab9e470e0c2c0..8b41df6b269f9 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -339,9 +339,12 @@ public void showMemoryUsage() { MDC.of(LogKeys.MEMORY_SIZE, memoryNotAccountedFor), MDC.of(LogKeys.TASK_ATTEMPT_ID, taskAttemptId)); logger.info( - "{} bytes of memory are used for execution and {} bytes of memory are used for storage", + "{} bytes of memory are used for execution " + + "and {} bytes of memory are used for storage " + + "and {} bytes of unmanaged memory are used", MDC.of(LogKeys.EXECUTION_MEMORY_SIZE, memoryManager.executionMemoryUsed()), - MDC.of(LogKeys.STORAGE_MEMORY_SIZE, memoryManager.storageMemoryUsed())); + MDC.of(LogKeys.STORAGE_MEMORY_SIZE, memoryManager.storageMemoryUsed()), + MDC.of(LogKeys.MEMORY_SIZE, UnifiedMemoryManager$.MODULE$.getUnmanagedMemoryUsed())); } } diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index db51f14415e1a..6b278c47f32f1 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -273,6 +273,14 @@ object UnifiedMemoryManager extends Logging { // Atomic flag to ensure polling is only started once per JVM private val pollingStarted = new AtomicBoolean(false) + /** + * Returns the total unmanaged memory in bytes, including both + * on-heap unmanaged memory and off-heap unmanaged memory. + */ + private[spark] def getUnmanagedMemoryUsed: Long = { + UnifiedMemoryManager.unmanagedOnHeapUsed.get() + UnifiedMemoryManager.unmanagedOffHeapUsed.get() + } + /** * Register an unmanaged memory consumer to track its memory usage. * From c07c3339cfe62e9a0f04b869383824636c91b789 Mon Sep 17 00:00:00 2001 From: vinodkc Date: Tue, 4 Nov 2025 11:54:10 +0900 Subject: [PATCH 026/400] [SPARK-54049][BUILD] Shade com.google.thirdparty package to fix Guava class conflicts in spark 4.0 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We upgraded Guava from 14.0.1 to 30+ in  spark 4.0 . Guava 33.4.0 used in Spark 4 consists of two main packages: - `com.google.common` - `com.google.thirdparty` Prior to this PR, only the `com.google.common` package was shaded into the spark-network-common jar, while classes under `com.google.thirdparty` remained unshaded in the spark-network-common jar. This partial shading causes classloading conflicts and runtime errors when a downstream project depends on both Spark and its own version of Guava. Eg: calls to guava class `com.google.common.net.InternetDomainName` fails with the following error: ``` Caused by: java.lang.NoSuchFieldError: EXACT at com.google.common.net.InternetDomainName.findSuffixOfType(InternetDomainName.java:226) at com.google.common.net.InternetDomainName.publicSuffixIndex(InternetDomainName.java:185) at com.google.common.net.InternetDomainName.hasPublicSuffix(InternetDomainName.java:400) at com.eadx.Domain$.printDomainInfo(Domain.scala:16) at com.eadx.TestApp$.main(TestApp.scala:16) ``` **Root Cause**: `com.google.common.net.InternetDomainName` uses classes from `com.google.thirdparty.publicsuffix`. The classloader resolves `com.google.common.net.InternetDomainName` from the downstream Guava jar, while `com.google.thirdparty.publicsuffix.PublicSuffixPatterns` is loaded from Spark 4.x Guava classes, leading to binary incompatibility. Example diagnostic: ``` InternetDomainName → guava-32.0.0-jre.jar (target/.../guava-32.0.0-jre.jar) PublicSuffixPatterns → spark-network-common_2.13-4.0.0.jar (target/.../spark-network-common_2.13-4.0.0.jar) ``` ### What changes were proposed in this pull request? This PR ensures package `com.google.thirdparty` is also shaded and isolated under the sparkproject namespace in Spark, preventing downstream class conflicts and runtime errors. ### Why are the changes needed? These changes are necessary to prevent runtime errors and class conflicts for downstream projects that depend on both Spark and Guava by restoring proper isolation of shaded Guava classes in spark ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No new test cases added; used existing UT and IT. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52767 from vinodkc/br_shade_guava_thirdparty. Authored-by: vinodkc Signed-off-by: Hyukjin Kwon (cherry picked from commit c8cb3e7506c4f36e6abb931237774383e981425d) Signed-off-by: Hyukjin Kwon --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index b1c250a34c723..107def928cbbb 100644 --- a/pom.xml +++ b/pom.xml @@ -3148,6 +3148,10 @@ com.google.common ${spark.shade.packageName}.guava + + com.google.thirdparty + ${spark.shade.packageName}.guava.thirdparty + org.dmg.pmml ${spark.shade.packageName}.dmg.pmml From 8edd591e4ef162b06df954772f5d1f22f096875b Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 3 Nov 2025 20:28:37 -0800 Subject: [PATCH 027/400] [SPARK-54115][CORE][UI] Escalate display ordering priority of connect server operation threads in thread dump page ### What changes were proposed in this pull request? Escalate display ordering priority of connect server execution threads in the thread dump page by defining a custom `threadInfoOrdering`. For connect server runs in local deploy mode, tasks also run in driver, in driver thread dump page, task threads display first, then connect operation threads. For connect server runs in other deploy modes (YARN, K8s, Standalone), in driver thread dump page, connect operation threads display first. ### Why are the changes needed? Currently, Spark executor displays the task threads first on the thread dump page, this does improve the user experience in troubleshooting "task stuck" issues. There are a lot of similar stuck issues on the driver's side too, e.g. driver may be stuck at the HMS/HDFS RPC call on the query planning phase, displaying the connect operation threads at the top on the driver thread dump pages makes the users easy to diagnose driver stuck issues for the Connect server. ### Does this PR introduce _any_ user-facing change? Yes, it affects the live UI thread dump page. ### How was this patch tested? Add UT. Also manually tested. Start a connect server in local mode, and use a client to run some queries. Note, since it runs in local mode, the task also executes at the driver side, the page display task threads first, then connect operation threads. Xnip2025-10-31_18-14-16 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52816 from pan3793/SPARK-54115. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit bdac5cd192af7ce8477760f8300da95d04ef49cd) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/executor/Executor.scala | 7 ++- .../scala/org/apache/spark/util/Utils.scala | 47 ++++++++++++------- .../org/apache/spark/util/UtilsSuite.scala | 42 ++++++++++++++++- .../execution/ExecuteThreadRunner.scala | 3 +- 4 files changed, 78 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a14ba21a0c186..eb45e810d4255 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -40,6 +40,7 @@ import org.slf4j.{MDC => SLF4JMDC} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.Executor.TASK_THREAD_NAME_PREFIX import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ @@ -132,7 +133,7 @@ private[spark] class Executor( private[executor] val threadPool = { val threadFactory = new ThreadFactoryBuilder() .setDaemon(true) - .setNameFormat("Executor task launch worker-%d") + .setNameFormat(s"$TASK_THREAD_NAME_PREFIX-%d") .setThreadFactory((r: Runnable) => new UninterruptibleThread(r, "unused")) .build() Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] @@ -478,7 +479,7 @@ private[spark] class Executor( val taskId = taskDescription.taskId val taskName = taskDescription.name - val threadName = s"Executor task launch worker for $taskName" + val threadName = s"$TASK_THREAD_NAME_PREFIX for $taskName" val mdcProperties = taskDescription.properties.asScala .filter(_._1.startsWith("mdc.")).toSeq @@ -1316,6 +1317,8 @@ private[spark] class Executor( } private[spark] object Executor extends Logging { + val TASK_THREAD_NAME_PREFIX = "Executor task launch worker" + // This is reserved for internal use by components that need to read task properties before a // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be // used instead. diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8b1ea4d25592f..81e86c82211c4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -65,6 +65,7 @@ import org.slf4j.Logger import org.apache.spark.{SPARK_VERSION, _} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.Executor.TASK_THREAD_NAME_PREFIX import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ @@ -2086,27 +2087,39 @@ private[spark] object Utils } } + val CONNECT_EXECUTE_THREAD_PREFIX = "SparkConnectExecuteThread" + + private val threadInfoOrdering = Ordering.fromLessThan { + (threadTrace1: ThreadInfo, threadTrace2: ThreadInfo) => { + def priority(ti: ThreadInfo): Int = ti.getThreadName match { + case name if name.startsWith(TASK_THREAD_NAME_PREFIX) => 100 + case name if name.startsWith(CONNECT_EXECUTE_THREAD_PREFIX) => 80 + case _ => 0 + } + + val v1 = priority(threadTrace1) + val v2 = priority(threadTrace2) + if (v1 == v2) { + val name1 = threadTrace1.getThreadName.toLowerCase(Locale.ROOT) + val name2 = threadTrace2.getThreadName.toLowerCase(Locale.ROOT) + val nameCmpRes = name1.compareTo(name2) + if (nameCmpRes == 0) { + threadTrace1.getThreadId < threadTrace2.getThreadId + } else { + nameCmpRes < 0 + } + } else { + v1 > v2 + } + } + } + /** Return a thread dump of all threads' stacktraces. Used to capture dumps for the web UI */ def getThreadDump(): Array[ThreadStackTrace] = { // We need to filter out null values here because dumpAllThreads() may return null array // elements for threads that are dead / don't exist. - val threadInfos = ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).filter(_ != null) - threadInfos.sortWith { case (threadTrace1, threadTrace2) => - val v1 = if (threadTrace1.getThreadName.contains("Executor task launch")) 1 else 0 - val v2 = if (threadTrace2.getThreadName.contains("Executor task launch")) 1 else 0 - if (v1 == v2) { - val name1 = threadTrace1.getThreadName().toLowerCase(Locale.ROOT) - val name2 = threadTrace2.getThreadName().toLowerCase(Locale.ROOT) - val nameCmpRes = name1.compareTo(name2) - if (nameCmpRes == 0) { - threadTrace1.getThreadId < threadTrace2.getThreadId - } else { - nameCmpRes < 0 - } - } else { - v1 > v2 - } - }.map(threadInfoToThreadStackTrace) + ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).filter(_ != null) + .sorted(threadInfoOrdering).map(threadInfoToThreadStackTrace) } /** Return a heap dump. Used to capture dumps for the web UI */ diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 933b6fc39e913..d600260e9df2a 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import java.io._ +import java.lang.management.ThreadInfo import java.lang.reflect.Field import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} @@ -37,6 +38,9 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext import org.apache.hadoop.ipc.{CallerContext => HadoopCallerContext} import org.apache.logging.log4j.Level +import org.mockito.Mockito.doReturn +import org.scalatest.PrivateMethodTester +import org.scalatestplus.mockito.MockitoSugar.mock import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TaskContext} import org.apache.spark.internal.config._ @@ -47,7 +51,7 @@ import org.apache.spark.scheduler.SparkListener import org.apache.spark.util.collection.Utils.createArray import org.apache.spark.util.io.ChunkedByteBufferInputStream -class UtilsSuite extends SparkFunSuite with ResetSystemProperties { +class UtilsSuite extends SparkFunSuite with ResetSystemProperties with PrivateMethodTester { test("timeConversion") { // Test -1 @@ -1126,6 +1130,42 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties { assert(pValue > threshold) } + test("ThreadInfoOrdering") { + val task1T = mock[ThreadInfo] + doReturn(11L).when(task1T).getThreadId + doReturn("Executor task launch worker for task 1.0 in stage 1.0 (TID 11)") + .when(task1T).getThreadName + doReturn("Executor task launch worker for task 1.0 in stage 1.0 (TID 11)") + .when(task1T).toString + + val task2T = mock[ThreadInfo] + doReturn(12L).when(task2T).getThreadId + doReturn("Executor task launch worker for task 2.0 in stage 1.0 (TID 22)") + .when(task2T).getThreadName + doReturn("Executor task launch worker for task 2.0 in stage 1.0 (TID 22)") + .when(task2T).toString + + val connectExecuteOp1T = mock[ThreadInfo] + doReturn(21L).when(connectExecuteOp1T).getThreadId + doReturn("SparkConnectExecuteThread_opId=16148fb4-4189-43c3-b8d4-8b3b6ddd41c7") + .when(connectExecuteOp1T).getThreadName + doReturn("SparkConnectExecuteThread_opId=16148fb4-4189-43c3-b8d4-8b3b6ddd41c7") + .when(connectExecuteOp1T).toString + + val connectExecuteOp2T = mock[ThreadInfo] + doReturn(22L).when(connectExecuteOp2T).getThreadId + doReturn("SparkConnectExecuteThread_opId=4e4d1cac-ffde-46c1-b7c2-808b726cb47e") + .when(connectExecuteOp2T).getThreadName + doReturn("SparkConnectExecuteThread_opId=4e4d1cac-ffde-46c1-b7c2-808b726cb47e") + .when(connectExecuteOp2T).toString + + val threadInfoOrderingMethod = + PrivateMethod[Ordering[ThreadInfo]](Symbol("threadInfoOrdering")) + val sorted = Seq(connectExecuteOp1T, connectExecuteOp2T, task1T, task2T) + .sorted(Utils.invokePrivate(threadInfoOrderingMethod())) + assert(sorted === Seq(task1T, task2T, connectExecuteOp1T, connectExecuteOp2T)) + } + test("redact sensitive information") { val sparkConf = new SparkConf diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index 38ed2528cbde0..f206ee1555a73 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteSessionTag, S import org.apache.spark.sql.connect.utils.ErrorUtils import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils +import org.apache.spark.util.Utils.CONNECT_EXECUTE_THREAD_PREFIX /** * This class launches the actual execution in an execution thread. The execution pushes the @@ -329,7 +330,7 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends } private class ExecutionThread() - extends Thread(s"SparkConnectExecuteThread_opId=${executeHolder.operationId}") { + extends Thread(s"${CONNECT_EXECUTE_THREAD_PREFIX}_opId=${executeHolder.operationId}") { override def run(): Unit = execute() } } From d0680d61e53627f8b00dee8d06c4512bbb9f8829 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Tue, 4 Nov 2025 17:13:50 +0900 Subject: [PATCH 028/400] [SPARK-54142][GEO][SQL][PYTHON] Implement the st_srid function in Scala and PySpark ### What changes were proposed in this pull request? Implement the `st_srid` function in Scala and PySpark API. ### Why are the changes needed? Expand API support for the `ST_Srid` expression. ### Does this PR introduce _any_ user-facing change? Yes, the new function is now available in Scala and PySpark API. ### How was this patch tested? Added appropriate Scala function unit tests: - `STFunctionsSuite` Added appropriate PySpark function unit tests: - `test_functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52841 from uros-db/geo-ST_Srid-scala. Authored-by: Uros Bojanic Signed-off-by: Hyukjin Kwon (cherry picked from commit 8c76795fa958288cb6b6bb5405b9973be6cf3ad5) Signed-off-by: Hyukjin Kwon --- .../reference/pyspark.sql/functions.rst | 1 + .../pyspark/sql/connect/functions/builtin.py | 7 ++++++ python/pyspark/sql/functions/__init__.py | 1 + python/pyspark/sql/functions/builtin.py | 25 +++++++++++++++++++ python/pyspark/sql/tests/test_functions.py | 15 +++++++++++ .../org/apache/spark/sql/functions.scala | 9 +++++++ .../apache/spark/sql/STFunctionsSuite.scala | 16 ++++++++++++ 7 files changed, 74 insertions(+) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index 6576c7245e311..edd87c26dbb6b 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -660,6 +660,7 @@ Geospatial ST Functions st_asbinary st_geogfromwkb st_geomfromwkb + st_srid UDF, UDTF and UDT diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 2c58ed946a82c..68caeef6ace83 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -4807,6 +4807,13 @@ def st_geomfromwkb(wkb: "ColumnOrName") -> Column: st_geomfromwkb.__doc__ = pysparkfuncs.st_geomfromwkb.__doc__ +def st_srid(geo: "ColumnOrName") -> Column: + return _invoke_function_over_columns("st_srid", geo) + + +st_srid.__doc__ = pysparkfuncs.st_srid.__doc__ + + # Call Functions diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index df9594f18c96d..fa579a222efad 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -526,6 +526,7 @@ "st_asbinary", "st_geogfromwkb", "st_geomfromwkb", + "st_srid", # Call Functions "call_udf", "pandas_udf", diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index ade6723485e26..26c498dfa252a 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -25971,6 +25971,31 @@ def st_geomfromwkb(wkb: "ColumnOrName") -> Column: return _invoke_function_over_columns("st_geomfromwkb", wkb) +@_try_remote_functions +def st_srid(geo: "ColumnOrName") -> Column: + """Returns the SRID of the input GEOGRAPHY or GEOMETRY value. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + geo : :class:`~pyspark.sql.Column` or str + A geospatial value, either a GEOGRAPHY or a GEOMETRY. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.st_srid(sf.st_geogfromwkb('wkb')).alias('result')).collect() + [Row(result=4326)] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.st_srid(sf.st_geomfromwkb('wkb')).alias('result')).collect() + [Row(result=0)] + """ + return _invoke_function_over_columns("st_srid", geo) + + # Call Functions diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 6dc0770d3df41..07c0c9aaf5242 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -2822,6 +2822,21 @@ def test_st_asbinary(self): ) self.assertEqual(results, [expected]) + def test_st_srid(self): + df = self.spark.createDataFrame( + [(bytes.fromhex("0101000000000000000000F03F0000000000000040"),)], + ["wkb"], + ) + results = df.select( + F.st_srid(F.st_geogfromwkb("wkb")), + F.st_srid(F.st_geomfromwkb("wkb")), + ).collect() + expected = Row( + 4326, + 0, + ) + self.assertEqual(results, [expected]) + class FunctionsTests(ReusedSQLTestCase, FunctionsTestsMixin): pass diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index ef78c842c5440..ab883d5933cf5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -9171,6 +9171,15 @@ object functions { def st_geomfromwkb(wkb: Column): Column = Column.fn("st_geomfromwkb", wkb) + /** + * Returns the SRID of the input GEOGRAPHY or GEOMETRY value. + * + * @group st_funcs + * @since 4.1.0 + */ + def st_srid(geo: Column): Column = + Column.fn("st_srid", geo) + ////////////////////////////////////////////////////////////////////////////////////////////// // Scala UDF functions ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala index 31a12d15fd349..6aee2cfa0776b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala @@ -42,4 +42,20 @@ class STFunctionsSuite extends QueryTest with SharedSparkSession { "0101000000000000000000f03f0000000000000040")) } + /** ST accessor expressions. */ + + test("st_srid") { + // Test data: Well-Known Binary (WKB) representations. + val df = Seq[(String)]( + ( + "0101000000000000000000f03f0000000000000040" + )).toDF("wkb") + // ST_GeogFromWKB/ST_GeomFromWKB and ST_Srid. + checkAnswer( + df.select( + st_srid(st_geogfromwkb(unhex($"wkb"))).as("col0"), + st_srid(st_geomfromwkb(unhex($"wkb"))).as("col1")), + Row(4326, 0)) + } + } From 783385065f88ed39dc3759acdea5a7f21a5aadf8 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 4 Nov 2025 09:18:19 -0800 Subject: [PATCH 029/400] [SPARK-54177][BUILD] Upgrade gRPC to 1.76 and protobuf to 6.33 ### What changes were proposed in this pull request? Bump gRPC from 1.67 to 1.76, with additional Python package upgrades for consistency: - `googleapis-common-protos==1.71.0` - `protobuf==6.33.0` And `buf v33.0` Fix the shading leaks of the `spark-connect` jar before ``` $ jar tf spark-connect_2.13-4.1.0-preview3.jar | grep '.class$' | grep -v 'org/apache/spark' | grep -v 'org/sparkproject' | grep -v 'META-INF' javax/annotation/Generated.class ... javax/ejb/EJB.class ... javax/persistence/PersistenceContext.class ... javax/xml/ws/WebServiceRef.class ... com/google/shopping/type/Price$Builder.class ... com/google/apps/card/v1/Widget$DataCase.class ... ``` after ``` $ jar tf spark-connect_2.13-4.2.0-SNAPSHOT.jar | grep '.class$' | grep -v 'org/apache/spark' | grep -v 'org/sparkproject' | grep -v 'META-INF' ``` ### Why are the changes needed? For Python: - [grpcio v1.75.1](https://github.com/grpc/grpc/releases/tag/v1.75.1) addes official Python 3.14 support - googleapis-common-proto v1.71.0 addes official Python 3.14 support, see https://github.com/googleapis/google-cloud-python/pull/14699 For Java: - v1.74 removes dependency on Tomcat's annotation API, see https://github.com/grpc/grpc-java/issues/9179 Check full release notes at: https://github.com/grpc/grpc/releases ### Does this PR introduce _any_ user-facing change? Maybe, reduce the potential conflict risks between Spark and user classes. ### How was this patch tested? Pass GHA, plus manual checks (see above sections). ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52874 from pan3793/SPARK-54177. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit e55333c9506b78e9746ed4b34e066838f81088e2) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 4 +-- .github/workflows/build_python_connect.yml | 2 +- .github/workflows/maven_test.yml | 2 +- .github/workflows/pages.yml | 2 +- .../workflows/python_hosted_runner_test.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 4 +-- dev/infra/Dockerfile | 4 +-- dev/requirements.txt | 8 +++--- dev/spark-test-image/docs/Dockerfile | 2 +- dev/spark-test-image/lint/Dockerfile | 4 +-- dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- .../python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- .../python-minimum/Dockerfile | 2 +- .../python-ps-minimum/Dockerfile | 2 +- pom.xml | 3 +-- project/SparkBuild.scala | 4 +-- .../docs/source/getting_started/install.rst | 12 ++++----- python/packaging/classic/setup.py | 4 +-- python/packaging/client/setup.py | 4 +-- python/packaging/connect/setup.py | 4 +-- python/pyspark/sql/connect/proto/base_pb2.py | 4 +-- .../pyspark/sql/connect/proto/catalog_pb2.py | 4 +-- .../pyspark/sql/connect/proto/commands_pb2.py | 4 +-- .../pyspark/sql/connect/proto/common_pb2.py | 4 +-- .../sql/connect/proto/example_plugins_pb2.py | 4 +-- .../sql/connect/proto/expressions_pb2.py | 4 +-- .../sql/connect/proto/ml_common_pb2.py | 4 +-- python/pyspark/sql/connect/proto/ml_pb2.py | 4 +-- .../sql/connect/proto/pipelines_pb2.py | 4 +-- .../sql/connect/proto/relations_pb2.py | 4 +-- python/pyspark/sql/connect/proto/types_pb2.py | 4 +-- .../sql/streaming/proto/StateMessage_pb2.py | 8 +++--- sql/connect/client/jdbc/pom.xml | 4 --- sql/connect/client/jvm/pom.xml | 4 --- sql/connect/common/pom.xml | 26 ------------------- sql/connect/common/src/main/buf.gen.yaml | 14 +++++----- sql/connect/server/pom.xml | 14 +++++----- sql/core/src/main/buf.gen.yaml | 2 +- 43 files changed, 83 insertions(+), 116 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 3ba71108f5533..2609f84d98cc3 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -361,7 +361,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') || contains(matrix.modules, 'yarn') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' python3.11 -m pip list # Run the tests. - name: Run tests @@ -765,7 +765,7 @@ jobs: python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.11 -m pip install 'black==23.12.1' 'protobuf==5.29.5' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip install 'black==23.12.1' 'protobuf==6.33.0' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' python3.11 -m pip list - name: Python CodeGen check for branch-3.5 if: inputs.branch == 'branch-3.5' diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index cec37af22dd7b..b1ebb45b9cbc1 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -72,7 +72,7 @@ jobs: python packaging/client/setup.py sdist cd dist pip install pyspark*client-*.tar.gz - pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting + pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting - name: List Python packages run: python -m pip list - name: Run tests (local) diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 95c9aac33fc6c..7bbfc420e02ab 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -175,7 +175,7 @@ jobs: - name: Install Python packages (Python 3.11) if: contains(matrix.modules, 'resource-managers#yarn') || (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' python3.11 -m pip list # Run the tests using script command. # BSD's script command doesn't support -c option, and the usage is different from Linux's one. diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index e800b40106ee3..2bba3dcaf176d 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -63,7 +63,7 @@ jobs: pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' - name: Install Ruby for documentation generation uses: ruby/setup-ruby@v1 diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index 77e85222c29dd..d55eb1d93799b 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -148,7 +148,7 @@ jobs: python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ - python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ + python${{matrix.python}} -m pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge - name: List Python packages run: python${{matrix.python}} -m pip list diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 86be7e0a82295..679998b893927 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -94,7 +94,7 @@ ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 @@ -111,7 +111,7 @@ RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PI RUN python3.10 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ -'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ +'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' RUN python3.10 -m pip list diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 1aa03735ce926..423b6ba820d07 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -97,7 +97,7 @@ RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.3' scipy coverage matp ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 @@ -149,7 +149,7 @@ RUN apt-get update && apt-get install -y \ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 # TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ +RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.76.0 grpcio-status==1.76.0 lxml jinja2 && \ python3.13 -m pip cache purge # Remove unused installation packages to free up disk space diff --git a/dev/requirements.txt b/dev/requirements.txt index 76652df744815..ddaeb9b3dd9db 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -61,10 +61,10 @@ black==23.12.1 py # Spark Connect (required) -grpcio>=1.67.0 -grpcio-status>=1.67.0 -googleapis-common-protos>=1.65.0 -protobuf==5.29.5 +grpcio>=1.76.0 +grpcio-status>=1.76.0 +googleapis-common-protos>=1.71.0 +protobuf==6.33.0 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 1c17ae122d638..e268ea7a8351b 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -91,6 +91,6 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 RUN python3.11 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ && python3.11 -m pip cache purge diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 07ff9c90b7591..6686e3808e035 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -82,8 +82,8 @@ RUN python3.11 -m pip install \ 'flake8==3.9.0' \ 'googleapis-common-protos-stubs==2.2.0' \ 'grpc-stubs==1.24.11' \ - 'grpcio-status==1.67.0' \ - 'grpcio==1.67.0' \ + 'grpcio-status==1.76.0' \ + 'grpcio==1.76.0' \ 'ipython' \ 'ipython_genutils' \ 'jinja2' \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index d0409e61a51a6..d33fb5f5d30e4 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=21.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index ce2ca23d18a64..46cfce36832bf 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 00fb7be788fd0..920f21bd47ee7 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 79cab824a5b21..db1039f5cb260 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index 031eb8772b59b..a50bf670b3f35 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.13 packages diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index abd5a7e01093d..f74c48bf346f8 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 0ba9b620bd8b0..7deb5e855319d 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index 122281ec0ea1d..ebafbc69ec4d1 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" # Install Python 3.9 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 680697c3f2d74..13a5f2db386c8 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/pom.xml b/pom.xml index 107def928cbbb..b428823b3cb51 100644 --- a/pom.xml +++ b/pom.xml @@ -305,9 +305,8 @@ 33.4.0-jre 1.0.2 - 1.67.1 + 1.76.0 1.1.4 - 6.0.53 4.0-10 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1d8de063133e3..253893cc225b5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -820,11 +820,13 @@ object SparkConnect { ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.checkerframework.@1").inAll, ShadeRule.rename("com.google.gson.**" -> "org.sparkproject.connect.gson.@1").inAll, ShadeRule.rename("com.google.api.**" -> "org.sparkproject.connect.google_protos.api.@1").inAll, + ShadeRule.rename("com.google.apps.**" -> "org.sparkproject.connect.google_protos.apps.@1").inAll, ShadeRule.rename("com.google.cloud.**" -> "org.sparkproject.connect.google_protos.cloud.@1").inAll, ShadeRule.rename("com.google.geo.**" -> "org.sparkproject.connect.google_protos.geo.@1").inAll, ShadeRule.rename("com.google.logging.**" -> "org.sparkproject.connect.google_protos.logging.@1").inAll, ShadeRule.rename("com.google.longrunning.**" -> "org.sparkproject.connect.google_protos.longrunning.@1").inAll, ShadeRule.rename("com.google.rpc.**" -> "org.sparkproject.connect.google_protos.rpc.@1").inAll, + ShadeRule.rename("com.google.shopping.**" -> "org.sparkproject.connect.google_protos.shopping.@1").inAll, ShadeRule.rename("com.google.type.**" -> "org.sparkproject.connect.google_protos.type.@1").inAll ), @@ -911,7 +913,6 @@ object SparkConnectJdbc { ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, - ShadeRule.rename("javax.annotation.**" -> "org.sparkproject.connect.client.javax.annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll @@ -991,7 +992,6 @@ object SparkConnectClient { ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, - ShadeRule.rename("javax.annotation.**" -> "org.sparkproject.connect.client.javax.annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index 82db489651ff9..8b3c969d756d9 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -227,9 +227,9 @@ Package Supported version Note ========================== ================= ========================== `pandas` >=2.2.0 Required for Spark Connect `pyarrow` >=15.0.0 Required for Spark Connect -`grpcio` >=1.67.0 Required for Spark Connect -`grpcio-status` >=1.67.0 Required for Spark Connect -`googleapis-common-protos` >=1.65.0 Required for Spark Connect +`grpcio` >=1.76.0 Required for Spark Connect +`grpcio-status` >=1.76.0 Required for Spark Connect +`googleapis-common-protos` >=1.71.0 Required for Spark Connect `graphviz` >=0.20 Optional for Spark Connect ========================== ================= ========================== @@ -310,9 +310,9 @@ Package Supported version Note ========================== ================= =================================================== `pandas` >=2.2.0 Required for Spark Connect and Spark SQL `pyarrow` >=15.0.0 Required for Spark Connect and Spark SQL -`grpcio` >=1.67.0 Required for Spark Connect -`grpcio-status` >=1.67.0 Required for Spark Connect -`googleapis-common-protos` >=1.65.0 Required for Spark Connect +`grpcio` >=1.76.0 Required for Spark Connect +`grpcio-status` >=1.76.0 Required for Spark Connect +`googleapis-common-protos` >=1.71.0 Required for Spark Connect `pyyaml` >=3.11 Required for spark-pipelines command line interface `graphviz` >=0.20 Optional for Spark Connect ========================== ================= =================================================== diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index eac97af2e8c89..e6ac729f20d6f 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -153,8 +153,8 @@ def _supports_symlinks(): _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "15.0.0" -_minimum_grpc_version = "1.67.0" -_minimum_googleapis_common_protos_version = "1.65.0" +_minimum_grpc_version = "1.76.0" +_minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index 7ec7e45a31604..c378d223cfcc3 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -136,8 +136,8 @@ _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "15.0.0" - _minimum_grpc_version = "1.67.0" - _minimum_googleapis_common_protos_version = "1.65.0" + _minimum_grpc_version = "1.76.0" + _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" with open("README.md") as f: diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index f2b53211b3a0d..3b88563bcfe7e 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -89,8 +89,8 @@ _minimum_pandas_version = "2.0.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "11.0.0" - _minimum_grpc_version = "1.67.0" - _minimum_googleapis_common_protos_version = "1.65.0" + _minimum_grpc_version = "1.76.0" + _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" with open("README.md") as f: diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 0fe992332de71..32bf6802df7bc 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/base.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/base.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/base.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.py b/python/pyspark/sql/connect/proto/catalog_pb2.py index 58c129a01daa8..054b367bd3b34 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.py +++ b/python/pyspark/sql/connect/proto/catalog_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/catalog.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/catalog.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/catalog.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index 694b4a9a9aa37..4eccf1b71706d 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/commands.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/commands.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/commands.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/common_pb2.py b/python/pyspark/sql/connect/proto/common_pb2.py index 07ea9f7ed3173..8abd8fa6dc041 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.py +++ b/python/pyspark/sql/connect/proto/common_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/common.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/common.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/common.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.py b/python/pyspark/sql/connect/proto/example_plugins_pb2.py index 71a73a6d592ae..423768ee63d65 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.py +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/example_plugins.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/example_plugins.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/example_plugins.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index bd75ade02d8be..0c466aeb67a0d 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/expressions.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/expressions.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/expressions.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.py b/python/pyspark/sql/connect/proto/ml_common_pb2.py index a49491b8ad1ed..de547fc2a102f 100644 --- a/python/pyspark/sql/connect/proto/ml_common_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/ml_common.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/ml_common.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/ml_common.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py index 9574966472a58..3bd141815c8eb 100644 --- a/python/pyspark/sql/connect/proto/ml_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/ml.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/ml.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/ml.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py index f3489f55ed874..d7321fa7cf0c1 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.py +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/pipelines.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/pipelines.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/pipelines.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index e7f319554c5e2..9e630b6ba5e4c 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/relations.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/relations.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/relations.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/types_pb2.py b/python/pyspark/sql/connect/proto/types_pb2.py index 9a52129103ad5..74efca8decf83 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.py +++ b/python/pyspark/sql/connect/proto/types_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/types.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/types.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/types.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py index b88fc2c5ca402..1305a6213c137 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: org/apache/spark/sql/execution/streaming/StateMessage.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -28,9 +28,9 @@ _runtime_version.ValidateProtobufRuntimeVersion( _runtime_version.Domain.PUBLIC, - 5, - 29, - 5, + 6, + 33, + 0, "", "org/apache/spark/sql/execution/streaming/StateMessage.proto", ) diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index c2dda12b1e639..15e3480f2e59d 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -181,10 +181,6 @@ org.checkerframework ${spark.shade.packageName}.org.checkerframework - - javax.annotation - ${spark.shade.packageName}.javax.annotation - io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 1c16b7e9ca8ca..c5e4e6a1adfda 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -225,10 +225,6 @@ org.checkerframework ${spark.shade.packageName}.org.checkerframework - - javax.annotation - ${spark.shade.packageName}.javax.annotation - io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 6ff47ec6d68ce..2713f23c51e6a 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -87,11 +87,6 @@ netty-transport-native-unix-common ${netty.version} - - org.apache.tomcat - annotations-api - ${tomcat.annotations.api.version} - - org.apache.tomcat - annotations-api - ${tomcat.annotations.api.version} - provided - org.scalacheck scalacheck_${scala.binary.version} @@ -376,6 +370,10 @@ com.google.api ${spark.shade.packageName}.connect.google_protos.api + + com.google.apps + ${spark.shade.packageName}.connect.google_protos.apps + com.google.cloud ${spark.shade.packageName}.connect.google_protos.cloud @@ -396,6 +394,10 @@ com.google.rpc ${spark.shade.packageName}.connect.google_protos.rpc + + com.google.shopping + ${spark.shade.packageName}.connect.google_protos.shopping + com.google.type ${spark.shade.packageName}.connect.google_protos.type diff --git a/sql/core/src/main/buf.gen.yaml b/sql/core/src/main/buf.gen.yaml index 01a34ed308444..5f87a840c6a49 100644 --- a/sql/core/src/main/buf.gen.yaml +++ b/sql/core/src/main/buf.gen.yaml @@ -17,7 +17,7 @@ version: v1 plugins: # Building the Python build and building the mypy interfaces. - - plugin: buf.build/protocolbuffers/python:v29.5 + - plugin: buf.build/protocolbuffers/python:v33.0 out: gen/proto/python - name: mypy out: gen/proto/python From 180759f6f2f936e813021cbebf2a280723841790 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Nov 2025 09:54:24 -0800 Subject: [PATCH 030/400] Revert "[SPARK-54177][BUILD] Upgrade gRPC to 1.76 and protobuf to 6.33" This reverts commit 783385065f88ed39dc3759acdea5a7f21a5aadf8. --- .github/workflows/build_and_test.yml | 4 +-- .github/workflows/build_python_connect.yml | 2 +- .github/workflows/maven_test.yml | 2 +- .github/workflows/pages.yml | 2 +- .../workflows/python_hosted_runner_test.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 4 +-- dev/infra/Dockerfile | 4 +-- dev/requirements.txt | 8 +++--- dev/spark-test-image/docs/Dockerfile | 2 +- dev/spark-test-image/lint/Dockerfile | 4 +-- dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- .../python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- .../python-minimum/Dockerfile | 2 +- .../python-ps-minimum/Dockerfile | 2 +- pom.xml | 3 ++- project/SparkBuild.scala | 4 +-- .../docs/source/getting_started/install.rst | 12 ++++----- python/packaging/classic/setup.py | 4 +-- python/packaging/client/setup.py | 4 +-- python/packaging/connect/setup.py | 4 +-- python/pyspark/sql/connect/proto/base_pb2.py | 4 +-- .../pyspark/sql/connect/proto/catalog_pb2.py | 4 +-- .../pyspark/sql/connect/proto/commands_pb2.py | 4 +-- .../pyspark/sql/connect/proto/common_pb2.py | 4 +-- .../sql/connect/proto/example_plugins_pb2.py | 4 +-- .../sql/connect/proto/expressions_pb2.py | 4 +-- .../sql/connect/proto/ml_common_pb2.py | 4 +-- python/pyspark/sql/connect/proto/ml_pb2.py | 4 +-- .../sql/connect/proto/pipelines_pb2.py | 4 +-- .../sql/connect/proto/relations_pb2.py | 4 +-- python/pyspark/sql/connect/proto/types_pb2.py | 4 +-- .../sql/streaming/proto/StateMessage_pb2.py | 8 +++--- sql/connect/client/jdbc/pom.xml | 4 +++ sql/connect/client/jvm/pom.xml | 4 +++ sql/connect/common/pom.xml | 26 +++++++++++++++++++ sql/connect/common/src/main/buf.gen.yaml | 14 +++++----- sql/connect/server/pom.xml | 14 +++++----- sql/core/src/main/buf.gen.yaml | 2 +- 43 files changed, 116 insertions(+), 83 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 2609f84d98cc3..3ba71108f5533 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -361,7 +361,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') || contains(matrix.modules, 'yarn') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' python3.11 -m pip list # Run the tests. - name: Run tests @@ -765,7 +765,7 @@ jobs: python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.11 -m pip install 'black==23.12.1' 'protobuf==6.33.0' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip install 'black==23.12.1' 'protobuf==5.29.5' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' python3.11 -m pip list - name: Python CodeGen check for branch-3.5 if: inputs.branch == 'branch-3.5' diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index b1ebb45b9cbc1..cec37af22dd7b 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -72,7 +72,7 @@ jobs: python packaging/client/setup.py sdist cd dist pip install pyspark*client-*.tar.gz - pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting + pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting - name: List Python packages run: python -m pip list - name: Run tests (local) diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 7bbfc420e02ab..95c9aac33fc6c 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -175,7 +175,7 @@ jobs: - name: Install Python packages (Python 3.11) if: contains(matrix.modules, 'resource-managers#yarn') || (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' python3.11 -m pip list # Run the tests using script command. # BSD's script command doesn't support -c option, and the usage is different from Linux's one. diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index 2bba3dcaf176d..e800b40106ee3 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -63,7 +63,7 @@ jobs: pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' - name: Install Ruby for documentation generation uses: ruby/setup-ruby@v1 diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index d55eb1d93799b..77e85222c29dd 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -148,7 +148,7 @@ jobs: python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ - python${{matrix.python}} -m pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' && \ + python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge - name: List Python packages run: python${{matrix.python}} -m pip list diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 679998b893927..86be7e0a82295 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -94,7 +94,7 @@ ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 @@ -111,7 +111,7 @@ RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PI RUN python3.10 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ -'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ +'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' RUN python3.10 -m pip list diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 423b6ba820d07..1aa03735ce926 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -97,7 +97,7 @@ RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.3' scipy coverage matp ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 @@ -149,7 +149,7 @@ RUN apt-get update && apt-get install -y \ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 # TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.76.0 grpcio-status==1.76.0 lxml jinja2 && \ +RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ python3.13 -m pip cache purge # Remove unused installation packages to free up disk space diff --git a/dev/requirements.txt b/dev/requirements.txt index ddaeb9b3dd9db..76652df744815 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -61,10 +61,10 @@ black==23.12.1 py # Spark Connect (required) -grpcio>=1.76.0 -grpcio-status>=1.76.0 -googleapis-common-protos>=1.71.0 -protobuf==6.33.0 +grpcio>=1.67.0 +grpcio-status>=1.67.0 +googleapis-common-protos>=1.65.0 +protobuf==5.29.5 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index e268ea7a8351b..1c17ae122d638 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -91,6 +91,6 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 RUN python3.11 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ && python3.11 -m pip cache purge diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 6686e3808e035..07ff9c90b7591 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -82,8 +82,8 @@ RUN python3.11 -m pip install \ 'flake8==3.9.0' \ 'googleapis-common-protos-stubs==2.2.0' \ 'grpc-stubs==1.24.11' \ - 'grpcio-status==1.76.0' \ - 'grpcio==1.76.0' \ + 'grpcio-status==1.67.0' \ + 'grpcio==1.67.0' \ 'ipython' \ 'ipython_genutils' \ 'jinja2' \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index d33fb5f5d30e4..d0409e61a51a6 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=21.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index 46cfce36832bf..ce2ca23d18a64 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 920f21bd47ee7..00fb7be788fd0 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index db1039f5cb260..79cab824a5b21 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index a50bf670b3f35..031eb8772b59b 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.13 packages diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index f74c48bf346f8..abd5a7e01093d 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 7deb5e855319d..0ba9b620bd8b0 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index ebafbc69ec4d1..122281ec0ea1d 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" # Install Python 3.9 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 13a5f2db386c8..680697c3f2d74 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/pom.xml b/pom.xml index b428823b3cb51..107def928cbbb 100644 --- a/pom.xml +++ b/pom.xml @@ -305,8 +305,9 @@ 33.4.0-jre 1.0.2 - 1.76.0 + 1.67.1 1.1.4 + 6.0.53 4.0-10 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 253893cc225b5..1d8de063133e3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -820,13 +820,11 @@ object SparkConnect { ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.checkerframework.@1").inAll, ShadeRule.rename("com.google.gson.**" -> "org.sparkproject.connect.gson.@1").inAll, ShadeRule.rename("com.google.api.**" -> "org.sparkproject.connect.google_protos.api.@1").inAll, - ShadeRule.rename("com.google.apps.**" -> "org.sparkproject.connect.google_protos.apps.@1").inAll, ShadeRule.rename("com.google.cloud.**" -> "org.sparkproject.connect.google_protos.cloud.@1").inAll, ShadeRule.rename("com.google.geo.**" -> "org.sparkproject.connect.google_protos.geo.@1").inAll, ShadeRule.rename("com.google.logging.**" -> "org.sparkproject.connect.google_protos.logging.@1").inAll, ShadeRule.rename("com.google.longrunning.**" -> "org.sparkproject.connect.google_protos.longrunning.@1").inAll, ShadeRule.rename("com.google.rpc.**" -> "org.sparkproject.connect.google_protos.rpc.@1").inAll, - ShadeRule.rename("com.google.shopping.**" -> "org.sparkproject.connect.google_protos.shopping.@1").inAll, ShadeRule.rename("com.google.type.**" -> "org.sparkproject.connect.google_protos.type.@1").inAll ), @@ -913,6 +911,7 @@ object SparkConnectJdbc { ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, + ShadeRule.rename("javax.annotation.**" -> "org.sparkproject.connect.client.javax.annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll @@ -992,6 +991,7 @@ object SparkConnectClient { ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, + ShadeRule.rename("javax.annotation.**" -> "org.sparkproject.connect.client.javax.annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index 8b3c969d756d9..82db489651ff9 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -227,9 +227,9 @@ Package Supported version Note ========================== ================= ========================== `pandas` >=2.2.0 Required for Spark Connect `pyarrow` >=15.0.0 Required for Spark Connect -`grpcio` >=1.76.0 Required for Spark Connect -`grpcio-status` >=1.76.0 Required for Spark Connect -`googleapis-common-protos` >=1.71.0 Required for Spark Connect +`grpcio` >=1.67.0 Required for Spark Connect +`grpcio-status` >=1.67.0 Required for Spark Connect +`googleapis-common-protos` >=1.65.0 Required for Spark Connect `graphviz` >=0.20 Optional for Spark Connect ========================== ================= ========================== @@ -310,9 +310,9 @@ Package Supported version Note ========================== ================= =================================================== `pandas` >=2.2.0 Required for Spark Connect and Spark SQL `pyarrow` >=15.0.0 Required for Spark Connect and Spark SQL -`grpcio` >=1.76.0 Required for Spark Connect -`grpcio-status` >=1.76.0 Required for Spark Connect -`googleapis-common-protos` >=1.71.0 Required for Spark Connect +`grpcio` >=1.67.0 Required for Spark Connect +`grpcio-status` >=1.67.0 Required for Spark Connect +`googleapis-common-protos` >=1.65.0 Required for Spark Connect `pyyaml` >=3.11 Required for spark-pipelines command line interface `graphviz` >=0.20 Optional for Spark Connect ========================== ================= =================================================== diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index e6ac729f20d6f..eac97af2e8c89 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -153,8 +153,8 @@ def _supports_symlinks(): _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "15.0.0" -_minimum_grpc_version = "1.76.0" -_minimum_googleapis_common_protos_version = "1.71.0" +_minimum_grpc_version = "1.67.0" +_minimum_googleapis_common_protos_version = "1.65.0" _minimum_pyyaml_version = "3.11" diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index c378d223cfcc3..7ec7e45a31604 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -136,8 +136,8 @@ _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "15.0.0" - _minimum_grpc_version = "1.76.0" - _minimum_googleapis_common_protos_version = "1.71.0" + _minimum_grpc_version = "1.67.0" + _minimum_googleapis_common_protos_version = "1.65.0" _minimum_pyyaml_version = "3.11" with open("README.md") as f: diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 3b88563bcfe7e..f2b53211b3a0d 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -89,8 +89,8 @@ _minimum_pandas_version = "2.0.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "11.0.0" - _minimum_grpc_version = "1.76.0" - _minimum_googleapis_common_protos_version = "1.71.0" + _minimum_grpc_version = "1.67.0" + _minimum_googleapis_common_protos_version = "1.65.0" _minimum_pyyaml_version = "3.11" with open("README.md") as f: diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 32bf6802df7bc..0fe992332de71 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/base.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/base.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/base.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.py b/python/pyspark/sql/connect/proto/catalog_pb2.py index 054b367bd3b34..58c129a01daa8 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.py +++ b/python/pyspark/sql/connect/proto/catalog_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/catalog.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/catalog.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/catalog.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index 4eccf1b71706d..694b4a9a9aa37 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/commands.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/commands.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/commands.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/common_pb2.py b/python/pyspark/sql/connect/proto/common_pb2.py index 8abd8fa6dc041..07ea9f7ed3173 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.py +++ b/python/pyspark/sql/connect/proto/common_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/common.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/common.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/common.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.py b/python/pyspark/sql/connect/proto/example_plugins_pb2.py index 423768ee63d65..71a73a6d592ae 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.py +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/example_plugins.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/example_plugins.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/example_plugins.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index 0c466aeb67a0d..bd75ade02d8be 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/expressions.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/expressions.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/expressions.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.py b/python/pyspark/sql/connect/proto/ml_common_pb2.py index de547fc2a102f..a49491b8ad1ed 100644 --- a/python/pyspark/sql/connect/proto/ml_common_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/ml_common.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/ml_common.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/ml_common.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py index 3bd141815c8eb..9574966472a58 100644 --- a/python/pyspark/sql/connect/proto/ml_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/ml.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/ml.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/ml.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py index d7321fa7cf0c1..f3489f55ed874 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.py +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/pipelines.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/pipelines.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/pipelines.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 9e630b6ba5e4c..e7f319554c5e2 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/relations.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/relations.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/relations.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/types_pb2.py b/python/pyspark/sql/connect/proto/types_pb2.py index 74efca8decf83..9a52129103ad5 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.py +++ b/python/pyspark/sql/connect/proto/types_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/types.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/types.proto" + _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/types.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py index 1305a6213c137..b88fc2c5ca402 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: org/apache/spark/sql/execution/streaming/StateMessage.proto -# Protobuf Python Version: 6.33.0 +# Protobuf Python Version: 5.29.5 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -28,9 +28,9 @@ _runtime_version.ValidateProtobufRuntimeVersion( _runtime_version.Domain.PUBLIC, - 6, - 33, - 0, + 5, + 29, + 5, "", "org/apache/spark/sql/execution/streaming/StateMessage.proto", ) diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index 15e3480f2e59d..c2dda12b1e639 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -181,6 +181,10 @@ org.checkerframework ${spark.shade.packageName}.org.checkerframework + + javax.annotation + ${spark.shade.packageName}.javax.annotation + io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index c5e4e6a1adfda..1c16b7e9ca8ca 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -225,6 +225,10 @@ org.checkerframework ${spark.shade.packageName}.org.checkerframework + + javax.annotation + ${spark.shade.packageName}.javax.annotation + io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 2713f23c51e6a..6ff47ec6d68ce 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -87,6 +87,11 @@ netty-transport-native-unix-common ${netty.version} + + org.apache.tomcat + annotations-api + ${tomcat.annotations.api.version} + + org.apache.tomcat + annotations-api + ${tomcat.annotations.api.version} + provided + org.scalacheck scalacheck_${scala.binary.version} @@ -370,10 +376,6 @@ com.google.api ${spark.shade.packageName}.connect.google_protos.api - - com.google.apps - ${spark.shade.packageName}.connect.google_protos.apps - com.google.cloud ${spark.shade.packageName}.connect.google_protos.cloud @@ -394,10 +396,6 @@ com.google.rpc ${spark.shade.packageName}.connect.google_protos.rpc - - com.google.shopping - ${spark.shade.packageName}.connect.google_protos.shopping - com.google.type ${spark.shade.packageName}.connect.google_protos.type diff --git a/sql/core/src/main/buf.gen.yaml b/sql/core/src/main/buf.gen.yaml index 5f87a840c6a49..01a34ed308444 100644 --- a/sql/core/src/main/buf.gen.yaml +++ b/sql/core/src/main/buf.gen.yaml @@ -17,7 +17,7 @@ version: v1 plugins: # Building the Python build and building the mypy interfaces. - - plugin: buf.build/protocolbuffers/python:v33.0 + - plugin: buf.build/protocolbuffers/python:v29.5 out: gen/proto/python - name: mypy out: gen/proto/python From f3055fd89296c923ed83fc28db07fe8399f07ef6 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Tue, 4 Nov 2025 10:02:36 -0800 Subject: [PATCH 031/400] [SPARK-54170][CORE] Use StructuredLogging message in Scala side ### What changes were proposed in this pull request? Uses StructuredLogging message in Scala side. ### Why are the changes needed? Follow-up of https://github.com/apache/spark/pull/52689. The StructuredLogging framework should be used. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52864 from ueshin/issues/SPARK-54170/mdc. Authored-by: Takuya Ueshin Signed-off-by: Dongjoon Hyun (cherry picked from commit b722b1c0d6f06df519281ee7e37292ea221b2f4d) Signed-off-by: Dongjoon Hyun --- .../java/org/apache/spark/internal/LogKeys.java | 1 + .../spark/api/python/PythonWorkerLogCapture.scala | 15 ++++++++++----- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java b/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java index 8b6d3614b86de..48bc6f201bc7f 100644 --- a/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java +++ b/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java @@ -614,6 +614,7 @@ public enum LogKeys implements LogKey { PYTHON_WORKER_CHANNEL_IS_BLOCKING_MODE, PYTHON_WORKER_CHANNEL_IS_CONNECTED, PYTHON_WORKER_HAS_INPUTS, + PYTHON_WORKER_ID, PYTHON_WORKER_IDLE_TIMEOUT, PYTHON_WORKER_IS_ALIVE, PYTHON_WORKER_MODULE, diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerLogCapture.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerLogCapture.scala index 71fc00546ef6e..a2a7c5ea14513 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerLogCapture.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerLogCapture.scala @@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ import org.apache.spark.SparkEnv -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.storage.{PythonWorkerLogBlockIdGenerator, PythonWorkerLogLine, RollingLogWriter} /** @@ -64,7 +64,9 @@ private[python] class PythonWorkerLogCapture( writer.close() } catch { case e: Exception => - logWarning(s"Failed to close log writer for worker $workerId", e) + logWarning( + log"Failed to close log writer for worker ${MDC(LogKeys.PYTHON_WORKER_ID, workerId)}", + e) } } } @@ -73,12 +75,14 @@ private[python] class PythonWorkerLogCapture( * Closes all active worker log writers. */ def closeAllWriters(): Unit = { - workerLogWriters.values().asScala.foreach { case (writer, _) => + workerLogWriters.asScala.foreach { case (workerId, (writer, _)) => try { writer.close() } catch { case e: Exception => - logWarning("Failed to close log writer", e) + logWarning( + log"Failed to close log writer for worker ${MDC(LogKeys.PYTHON_WORKER_ID, workerId)}", + e) } } workerLogWriters.clear() @@ -128,7 +132,8 @@ private[python] class PythonWorkerLogCapture( } } catch { case e: Exception => - logWarning(s"Failed to write log for worker $workerId", e) + logWarning( + log"Failed to write log for worker ${MDC(LogKeys.PYTHON_WORKER_ID, workerId)}", e) } } prefix From 52fc0d4c681cbd66acd6bd1ebfc1facaff4c4eca Mon Sep 17 00:00:00 2001 From: EugenYushin Date: Tue, 4 Nov 2025 10:47:13 -0800 Subject: [PATCH 032/400] [SPARK-54056][SQL] resolve substitution for SQLConf settings in Catalogs ### What changes were proposed in this pull request? Pass `SQLConf` values through `ConfigReader` bindings, allowing for `${env:xyz} -> sys.env.get("xyz")` substitution before passing the settings to `CatalogPlugin` implementations. ### Why are the changes needed? Settings for custom (table) catalog are not being rendered the same way as any other `SQLConf` settings. Particular catalog can define its own set of properties along with custom name for that catalog, e.g. "spark.sql.catalog.test-name". This prevents from registering these properties using [SQLConf.buildConf](https://github.com/apache/spark/blob/54dee4a4b59cf3817f15da46d35f42a81a3b1c07/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L111) machinery, meaning that [ConfigReader bindings](https://github.com/apache/spark/blob/fe904e6973b7a8fdadc5e253a6a74e8ccb359287/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala#L54-L57) are being skipped during properties resolution flow. This becomes an issue if such props are set in config files like `spark-defaults.conf` and not through code (where we can access `sys.env` directly). ``` scala> spark.conf.set("spark.sql.catalog.mssql", "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog") scala> spark.conf.set("spark.sql.catalog.mssql.url", "jdbc:sqlserver://...") scala> spark.conf.set("spark.sql.catalog.mssql.user", "${env:MSSQL_USER}") scala> spark.conf.set("spark.sql.catalog.mssql.password", "${env:MSSQL_PASSWORD}") scala> spark.sql("show tables in mssql").show() com.microsoft.sqlserver.jdbc.SQLServerException: Login failed for user '${env:MSSQL_PASSWORD}'. ClientConnectionId:4f9efa06-90bb-4498-9099-b1d9d3fac935 at com.microsoft.sqlserver.jdbc.SQLServerException.makeFromDatabaseError(SQLServerException.java:278) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? * unit testing is tricky for this change cause' we need to deal with `sys.env` or `sys.props`. At the same time, `Catalogs` is object which doesn't allow us to override `ConfigReader` the same way its done in [ConfigReaderSuite](https://github.com/apache/spark/blob/b7763a7eae2b9609012cbb4ce981276c6471cc4e/core/src/test/scala/org/apache/spark/internal/config/ConfigReaderSuite.scala#L27-L31) * manual testing via `spark-shell` ``` export MSSQL_USER=... export MSSQL_PASSWORD=... ./bin/spark-shell --master local --driver-class-path mssql-jdbc-13.2.0.jre11.jar --jars mssql-jdbc-13.2.0.jre11.jar ... scala> sys.env.get("MSSQL_USER") val res0: Option[String] = Some(...) scala> spark.conf.set("spark.sql.catalog.mssql", "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog") scala> spark.conf.set("spark.sql.catalog.mssql.url", "jdbc:sqlserver://...") scala> spark.conf.set("spark.sql.catalog.mssql.user", "${env:MSSQL_USER}") scala> spark.conf.set("spark.sql.catalog.mssql.password", "${env:MSSQL_PASSWORD}") scala> spark.sql("show tables in mssql").show() +---------+-------------------+-----------+ |namespace| tableName|isTemporary| +---------+-------------------+-----------+ ... ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #52759 from EugeneYushin/catalogs-load-conf. Lead-authored-by: EugenYushin Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit a1428d40970fdf9f884b0d63d2f0aefd2a18bc4e) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/connector/catalog/Catalogs.scala | 8 +++++++- .../spark/sql/connector/catalog/CatalogLoadingSuite.java | 7 ++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala index 419191f8f9c00..e6c70fdabb159 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/Catalogs.scala @@ -22,6 +22,7 @@ import java.util import java.util.regex.Pattern import org.apache.spark.SparkException +import org.apache.spark.internal.config.ConfigReader import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -93,10 +94,15 @@ private[sql] object Catalogs { private def catalogOptions(name: String, conf: SQLConf) = { val prefix = Pattern.compile("^spark\\.sql\\.catalog\\." + name + "\\.(.+)") val options = new util.HashMap[String, String] + val reader = new ConfigReader(options) conf.getAllConfs.foreach { case (key, value) => val matcher = prefix.matcher(key) - if (matcher.matches && matcher.groupCount > 0) options.put(matcher.group(1), value) + if (matcher.matches && matcher.groupCount > 0) { + // pass config entries through default ConfigReader mechanics, + // substituting prefixes from bindings: ${env:XYZ} -> sys.env.get("XYZ") + options.put(matcher.group(1), reader.substitute(value)) + } } new CaseInsensitiveStringMap(options) } diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java index 339f16407ae60..c7e8d7b0f7f30 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector.catalog; +import org.apache.spark.network.util.JavaUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -24,6 +25,7 @@ import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.apache.spark.util.Utils; + public class CatalogLoadingSuite { @Test public void testLoad() throws SparkException { @@ -58,6 +60,7 @@ public void testInitializationOptions() throws SparkException { conf.setConfString("spark.sql.catalog.test-name", TestCatalogPlugin.class.getCanonicalName()); conf.setConfString("spark.sql.catalog.test-name.name", "not-catalog-name"); conf.setConfString("spark.sql.catalog.test-name.kEy", "valUE"); + conf.setConfString("spark.sql.catalog.test-name.osName", "${system:os.name}"); CatalogPlugin plugin = Catalogs.load("test-name", conf); Assertions.assertNotNull(plugin,"Should instantiate a non-null plugin"); @@ -66,11 +69,13 @@ public void testInitializationOptions() throws SparkException { TestCatalogPlugin testPlugin = (TestCatalogPlugin) plugin; - Assertions.assertEquals(2, testPlugin.options.size(), "Options should contain only two keys"); + Assertions.assertEquals(3, testPlugin.options.size(), "Options should contain only three keys"); Assertions.assertEquals("not-catalog-name", testPlugin.options.get("name"), "Options should contain correct value for name (not overwritten)"); Assertions.assertEquals("valUE", testPlugin.options.get("key"), "Options should contain correct value for key"); + Assertions.assertEquals(JavaUtils.osName, testPlugin.options.get("osName"), + "Options should contain correct substitution for value"); } @Test From 58ec60d26974242245ec22de5e70ccf09942c42e Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Tue, 4 Nov 2025 11:04:43 -0800 Subject: [PATCH 033/400] [SPARK-38498][DSTREAM] Support customized StreamingListener by configuration ### What changes were proposed in this pull request? Currently, if user want to add an customized StreamingListener to StreamingContext, we need to add this listener in customized code. ``` streamingContext.addStreamingListener() ``` In this pr, we can define customized StreamingListener by set conf ``` spark.streaming.extraListeners ``` And it can support two constructor 1. No construct parameter 2. one constructor parameter of `SparkConf` ### Why are the changes needed? Some time we want to add some common StreamingListener to do some customized analysis, it is noisy to do this for all job, with this configuration, we can do this together just by setting a common env. ### Does this PR introduce _any_ user-facing change? User can set StreamingContext by set configuration ``` spark.streaming.extraListeners ``` ### How was this patch tested? Added UT Closes #35799 from AngersZhuuuu/SPARK-38498. Authored-by: Angerszhuuuu Signed-off-by: Wenchen Fan (cherry picked from commit 8100636ab8707d068b02b9e7cd3bfe1b850ff86e) Signed-off-by: Wenchen Fan --- .../spark/streaming/StreamingConf.scala | 7 +++++ .../spark/streaming/StreamingContext.scala | 28 ++++++++++++++++--- .../streaming/StreamingListenerSuite.scala | 27 +++++++++++++++++- 3 files changed, 57 insertions(+), 5 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala index bb80bd7072e8e..39f3d495bdbf2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala @@ -185,4 +185,11 @@ object StreamingConf { .longConf .createWithDefault(0) + private[streaming] val STREAMING_EXTRA_LISTENERS = + ConfigBuilder("spark.streaming.extraListeners") + .doc("Class names of streaming listeners to add to StreamingContext during initialization.") + .version("4.1.0") + .stringConf + .toSequence + .createOptional } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 79bc38318f919..139b83ba0d079 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -41,12 +41,11 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.SerializationDebugger import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingConf.STOP_GRACEFULLY_ON_SHUTDOWN +import org.apache.spark.streaming.StreamingConf.{STOP_GRACEFULLY_ON_SHUTDOWN, STREAMING_EXTRA_LISTENERS} import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.streaming.scheduler. - {ExecutorAllocationManager, JobScheduler, StreamingListener, StreamingListenerStreamingStarted} +import org.apache.spark.streaming.scheduler.{ExecutorAllocationManager, JobScheduler, StreamingListener, StreamingListenerStreamingStarted} import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} import org.apache.spark.util.{CallSite, ShutdownHookManager, ThreadUtils, Utils} @@ -584,7 +583,7 @@ class StreamingContext private[streaming] ( validate() registerProgressListener() - + registerExtraStreamingListener() // Start the streaming scheduler in a new thread, so that thread local properties // like call sites and job groups can be reset without affecting those of the // current thread. @@ -622,6 +621,27 @@ class StreamingContext private[streaming] ( } } + /** + * Registers streaming listeners specified in spark.streaming.extraListeners. + */ + private def registerExtraStreamingListener(): Unit = { + try { + conf.get(STREAMING_EXTRA_LISTENERS).foreach { classNames => + val listeners = Utils.loadExtensions(classOf[StreamingListener], classNames, conf) + listeners.foreach { listener => + addStreamingListener(listener) + logInfo(s"Registered streaming listener ${listener.getClass().getName()}") + } + } + } catch { + case e: Exception => + try { + stop() + } finally { + throw new SparkException(s"Exception when registering StreamingListener", e) + } + } + } /** * Wait for the execution to stop. Any exceptions that occurs during the execution diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 63899f961e7b0..fa560406ac3f0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -32,8 +32,9 @@ import org.scalatest.matchers.must.Matchers import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver @@ -235,6 +236,19 @@ class StreamingListenerSuite extends TestSuiteBase with LocalStreamingContext wi verifyNoMoreInteractions(streamingListener) } + test("SPARK-38498: Support extra streaming listener") { + val conf = new SparkConf().setMaster("local").setAppName("customized streaming listener") + .set(UI.UI_ENABLED, false) + .set(StreamingConf.STREAMING_EXTRA_LISTENERS.key, + classOf[ExtraStreamingListener].getName) + val sc = new SparkContext(conf) + ssc = new StreamingContext(sc, Milliseconds(1000)) + val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) + inputStream.foreachRDD(_.count()) + startStreamingContextAndCallStop(ssc) + assert(ExtraStreamingListenerBatchCounter.COMPLETED_BATCH >= 1) + } + private def startStreamingContextAndCallStop(_ssc: StreamingContext): Unit = { val contextStoppingCollector = new StreamingContextStoppingCollector(_ssc) _ssc.addStreamingListener(contextStoppingCollector) @@ -386,3 +400,14 @@ class StreamingContextStoppingCollector(val ssc: StreamingContext) extends Strea } } } + +class ExtraStreamingListener extends StreamingListener { + import ExtraStreamingListenerBatchCounter._ + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + COMPLETED_BATCH += 1 + } +} + +object ExtraStreamingListenerBatchCounter { + var COMPLETED_BATCH = 0 +} From d548120eda7b0947cb47e22b93fb457a010eabf4 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Tue, 4 Nov 2025 14:15:54 -0800 Subject: [PATCH 034/400] [SPARK-54175][GEO][CONNECT] Add Geography and Geometry types to Spark Connect proto ### What changes were proposed in this pull request? Add `Geography` and `Geometry` types to Spark Connect proto. ### Why are the changes needed? Enabling geospatial types in Spark Connect. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Regenerated the corresponding Spark Connect proto files. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52872 from uros-db/geo-spark-connect-proto. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit 32060bf99fc834e197f3e39d8530cd97a39df4ff) Signed-off-by: Wenchen Fan --- python/pyspark/sql/connect/proto/types_pb2.py | 116 +++++++++--------- .../pyspark/sql/connect/proto/types_pb2.pyi | 59 +++++++++ .../main/protobuf/spark/connect/types.proto | 18 ++- 3 files changed, 134 insertions(+), 59 deletions(-) diff --git a/python/pyspark/sql/connect/proto/types_pb2.py b/python/pyspark/sql/connect/proto/types_pb2.py index 9a52129103ad5..4e35f6b8911a2 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.py +++ b/python/pyspark/sql/connect/proto/types_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b"\n\x19spark/connect/types.proto\x12\rspark.connect\"\xac#\n\x08\x44\x61taType\x12\x32\n\x04null\x18\x01 \x01(\x0b\x32\x1c.spark.connect.DataType.NULLH\x00R\x04null\x12\x38\n\x06\x62inary\x18\x02 \x01(\x0b\x32\x1e.spark.connect.DataType.BinaryH\x00R\x06\x62inary\x12;\n\x07\x62oolean\x18\x03 \x01(\x0b\x32\x1f.spark.connect.DataType.BooleanH\x00R\x07\x62oolean\x12\x32\n\x04\x62yte\x18\x04 \x01(\x0b\x32\x1c.spark.connect.DataType.ByteH\x00R\x04\x62yte\x12\x35\n\x05short\x18\x05 \x01(\x0b\x32\x1d.spark.connect.DataType.ShortH\x00R\x05short\x12;\n\x07integer\x18\x06 \x01(\x0b\x32\x1f.spark.connect.DataType.IntegerH\x00R\x07integer\x12\x32\n\x04long\x18\x07 \x01(\x0b\x32\x1c.spark.connect.DataType.LongH\x00R\x04long\x12\x35\n\x05\x66loat\x18\x08 \x01(\x0b\x32\x1d.spark.connect.DataType.FloatH\x00R\x05\x66loat\x12\x38\n\x06\x64ouble\x18\t \x01(\x0b\x32\x1e.spark.connect.DataType.DoubleH\x00R\x06\x64ouble\x12;\n\x07\x64\x65\x63imal\x18\n \x01(\x0b\x32\x1f.spark.connect.DataType.DecimalH\x00R\x07\x64\x65\x63imal\x12\x38\n\x06string\x18\x0b \x01(\x0b\x32\x1e.spark.connect.DataType.StringH\x00R\x06string\x12\x32\n\x04\x63har\x18\x0c \x01(\x0b\x32\x1c.spark.connect.DataType.CharH\x00R\x04\x63har\x12<\n\x08var_char\x18\r \x01(\x0b\x32\x1f.spark.connect.DataType.VarCharH\x00R\x07varChar\x12\x32\n\x04\x64\x61te\x18\x0e \x01(\x0b\x32\x1c.spark.connect.DataType.DateH\x00R\x04\x64\x61te\x12\x41\n\ttimestamp\x18\x0f \x01(\x0b\x32!.spark.connect.DataType.TimestampH\x00R\ttimestamp\x12K\n\rtimestamp_ntz\x18\x10 \x01(\x0b\x32$.spark.connect.DataType.TimestampNTZH\x00R\x0ctimestampNtz\x12W\n\x11\x63\x61lendar_interval\x18\x11 \x01(\x0b\x32(.spark.connect.DataType.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12[\n\x13year_month_interval\x18\x12 \x01(\x0b\x32).spark.connect.DataType.YearMonthIntervalH\x00R\x11yearMonthInterval\x12U\n\x11\x64\x61y_time_interval\x18\x13 \x01(\x0b\x32'.spark.connect.DataType.DayTimeIntervalH\x00R\x0f\x64\x61yTimeInterval\x12\x35\n\x05\x61rray\x18\x14 \x01(\x0b\x32\x1d.spark.connect.DataType.ArrayH\x00R\x05\x61rray\x12\x38\n\x06struct\x18\x15 \x01(\x0b\x32\x1e.spark.connect.DataType.StructH\x00R\x06struct\x12/\n\x03map\x18\x16 \x01(\x0b\x32\x1b.spark.connect.DataType.MapH\x00R\x03map\x12;\n\x07variant\x18\x19 \x01(\x0b\x32\x1f.spark.connect.DataType.VariantH\x00R\x07variant\x12/\n\x03udt\x18\x17 \x01(\x0b\x32\x1b.spark.connect.DataType.UDTH\x00R\x03udt\x12>\n\x08unparsed\x18\x18 \x01(\x0b\x32 .spark.connect.DataType.UnparsedH\x00R\x08unparsed\x12\x32\n\x04time\x18\x1c \x01(\x0b\x32\x1c.spark.connect.DataType.TimeH\x00R\x04time\x1a\x43\n\x07\x42oolean\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x42yte\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05Short\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Integer\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04Long\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05\x46loat\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x42\n\x06\x44ouble\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a`\n\x06String\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x1c\n\tcollation\x18\x02 \x01(\tR\tcollation\x1a\x42\n\x06\x42inary\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04NULL\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x45\n\tTimestamp\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x44\x61te\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aH\n\x0cTimestampNTZ\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aq\n\x04Time\x12!\n\tprecision\x18\x01 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReferenceB\x0c\n\n_precision\x1aL\n\x10\x43\x61lendarInterval\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xb3\x01\n\x11YearMonthInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1a\xb1\x01\n\x0f\x44\x61yTimeInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1aX\n\x04\x43har\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a[\n\x07VarChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\x99\x01\n\x07\x44\x65\x63imal\x12\x19\n\x05scale\x18\x01 \x01(\x05H\x00R\x05scale\x88\x01\x01\x12!\n\tprecision\x18\x02 \x01(\x05H\x01R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x08\n\x06_scaleB\x0c\n\n_precision\x1a\xa1\x01\n\x0bStructField\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x34\n\tdata_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x08\x64\x61taType\x12\x1a\n\x08nullable\x18\x03 \x01(\x08R\x08nullable\x12\x1f\n\x08metadata\x18\x04 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x7f\n\x06Struct\x12;\n\x06\x66ields\x18\x01 \x03(\x0b\x32#.spark.connect.DataType.StructFieldR\x06\x66ields\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\xa2\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12#\n\rcontains_null\x18\x02 \x01(\x08R\x0c\x63ontainsNull\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x1a\xdb\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12.\n\x13value_contains_null\x18\x03 \x01(\x08R\x11valueContainsNull\x12\x38\n\x18type_variation_reference\x18\x04 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Variant\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xa1\x02\n\x03UDT\x12\x12\n\x04type\x18\x01 \x01(\tR\x04type\x12 \n\tjvm_class\x18\x02 \x01(\tH\x00R\x08jvmClass\x88\x01\x01\x12&\n\x0cpython_class\x18\x03 \x01(\tH\x01R\x0bpythonClass\x88\x01\x01\x12;\n\x17serialized_python_class\x18\x04 \x01(\tH\x02R\x15serializedPythonClass\x88\x01\x01\x12\x37\n\x08sql_type\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x07sqlType\x88\x01\x01\x42\x0c\n\n_jvm_classB\x0f\n\r_python_classB\x1a\n\x18_serialized_python_classB\x0b\n\t_sql_type\x1a\x34\n\x08Unparsed\x12(\n\x10\x64\x61ta_type_string\x18\x01 \x01(\tR\x0e\x64\x61taTypeStringB\x06\n\x04kindJ\x04\x08\x1a\x10\x1bJ\x04\x08\x1b\x10\x1c\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3" + b"\n\x19spark/connect/types.proto\x12\rspark.connect\"\xd8%\n\x08\x44\x61taType\x12\x32\n\x04null\x18\x01 \x01(\x0b\x32\x1c.spark.connect.DataType.NULLH\x00R\x04null\x12\x38\n\x06\x62inary\x18\x02 \x01(\x0b\x32\x1e.spark.connect.DataType.BinaryH\x00R\x06\x62inary\x12;\n\x07\x62oolean\x18\x03 \x01(\x0b\x32\x1f.spark.connect.DataType.BooleanH\x00R\x07\x62oolean\x12\x32\n\x04\x62yte\x18\x04 \x01(\x0b\x32\x1c.spark.connect.DataType.ByteH\x00R\x04\x62yte\x12\x35\n\x05short\x18\x05 \x01(\x0b\x32\x1d.spark.connect.DataType.ShortH\x00R\x05short\x12;\n\x07integer\x18\x06 \x01(\x0b\x32\x1f.spark.connect.DataType.IntegerH\x00R\x07integer\x12\x32\n\x04long\x18\x07 \x01(\x0b\x32\x1c.spark.connect.DataType.LongH\x00R\x04long\x12\x35\n\x05\x66loat\x18\x08 \x01(\x0b\x32\x1d.spark.connect.DataType.FloatH\x00R\x05\x66loat\x12\x38\n\x06\x64ouble\x18\t \x01(\x0b\x32\x1e.spark.connect.DataType.DoubleH\x00R\x06\x64ouble\x12;\n\x07\x64\x65\x63imal\x18\n \x01(\x0b\x32\x1f.spark.connect.DataType.DecimalH\x00R\x07\x64\x65\x63imal\x12\x38\n\x06string\x18\x0b \x01(\x0b\x32\x1e.spark.connect.DataType.StringH\x00R\x06string\x12\x32\n\x04\x63har\x18\x0c \x01(\x0b\x32\x1c.spark.connect.DataType.CharH\x00R\x04\x63har\x12<\n\x08var_char\x18\r \x01(\x0b\x32\x1f.spark.connect.DataType.VarCharH\x00R\x07varChar\x12\x32\n\x04\x64\x61te\x18\x0e \x01(\x0b\x32\x1c.spark.connect.DataType.DateH\x00R\x04\x64\x61te\x12\x41\n\ttimestamp\x18\x0f \x01(\x0b\x32!.spark.connect.DataType.TimestampH\x00R\ttimestamp\x12K\n\rtimestamp_ntz\x18\x10 \x01(\x0b\x32$.spark.connect.DataType.TimestampNTZH\x00R\x0ctimestampNtz\x12W\n\x11\x63\x61lendar_interval\x18\x11 \x01(\x0b\x32(.spark.connect.DataType.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12[\n\x13year_month_interval\x18\x12 \x01(\x0b\x32).spark.connect.DataType.YearMonthIntervalH\x00R\x11yearMonthInterval\x12U\n\x11\x64\x61y_time_interval\x18\x13 \x01(\x0b\x32'.spark.connect.DataType.DayTimeIntervalH\x00R\x0f\x64\x61yTimeInterval\x12\x35\n\x05\x61rray\x18\x14 \x01(\x0b\x32\x1d.spark.connect.DataType.ArrayH\x00R\x05\x61rray\x12\x38\n\x06struct\x18\x15 \x01(\x0b\x32\x1e.spark.connect.DataType.StructH\x00R\x06struct\x12/\n\x03map\x18\x16 \x01(\x0b\x32\x1b.spark.connect.DataType.MapH\x00R\x03map\x12;\n\x07variant\x18\x19 \x01(\x0b\x32\x1f.spark.connect.DataType.VariantH\x00R\x07variant\x12/\n\x03udt\x18\x17 \x01(\x0b\x32\x1b.spark.connect.DataType.UDTH\x00R\x03udt\x12>\n\x08geometry\x18\x1a \x01(\x0b\x32 .spark.connect.DataType.GeometryH\x00R\x08geometry\x12\x41\n\tgeography\x18\x1b \x01(\x0b\x32!.spark.connect.DataType.GeographyH\x00R\tgeography\x12>\n\x08unparsed\x18\x18 \x01(\x0b\x32 .spark.connect.DataType.UnparsedH\x00R\x08unparsed\x12\x32\n\x04time\x18\x1c \x01(\x0b\x32\x1c.spark.connect.DataType.TimeH\x00R\x04time\x1a\x43\n\x07\x42oolean\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x42yte\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05Short\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Integer\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04Long\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05\x46loat\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x42\n\x06\x44ouble\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a`\n\x06String\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x1c\n\tcollation\x18\x02 \x01(\tR\tcollation\x1a\x42\n\x06\x42inary\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04NULL\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x45\n\tTimestamp\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x44\x61te\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aH\n\x0cTimestampNTZ\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aq\n\x04Time\x12!\n\tprecision\x18\x01 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReferenceB\x0c\n\n_precision\x1aL\n\x10\x43\x61lendarInterval\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xb3\x01\n\x11YearMonthInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1a\xb1\x01\n\x0f\x44\x61yTimeInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1aX\n\x04\x43har\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a[\n\x07VarChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\x99\x01\n\x07\x44\x65\x63imal\x12\x19\n\x05scale\x18\x01 \x01(\x05H\x00R\x05scale\x88\x01\x01\x12!\n\tprecision\x18\x02 \x01(\x05H\x01R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x08\n\x06_scaleB\x0c\n\n_precision\x1a\xa1\x01\n\x0bStructField\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x34\n\tdata_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x08\x64\x61taType\x12\x1a\n\x08nullable\x18\x03 \x01(\x08R\x08nullable\x12\x1f\n\x08metadata\x18\x04 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x7f\n\x06Struct\x12;\n\x06\x66ields\x18\x01 \x03(\x0b\x32#.spark.connect.DataType.StructFieldR\x06\x66ields\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\xa2\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12#\n\rcontains_null\x18\x02 \x01(\x08R\x0c\x63ontainsNull\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x1a\xdb\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12.\n\x13value_contains_null\x18\x03 \x01(\x08R\x11valueContainsNull\x12\x38\n\x18type_variation_reference\x18\x04 \x01(\rR\x16typeVariationReference\x1aX\n\x08Geometry\x12\x12\n\x04srid\x18\x01 \x01(\x05R\x04srid\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1aY\n\tGeography\x12\x12\n\x04srid\x18\x01 \x01(\x05R\x04srid\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Variant\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xa1\x02\n\x03UDT\x12\x12\n\x04type\x18\x01 \x01(\tR\x04type\x12 \n\tjvm_class\x18\x02 \x01(\tH\x00R\x08jvmClass\x88\x01\x01\x12&\n\x0cpython_class\x18\x03 \x01(\tH\x01R\x0bpythonClass\x88\x01\x01\x12;\n\x17serialized_python_class\x18\x04 \x01(\tH\x02R\x15serializedPythonClass\x88\x01\x01\x12\x37\n\x08sql_type\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x07sqlType\x88\x01\x01\x42\x0c\n\n_jvm_classB\x0f\n\r_python_classB\x1a\n\x18_serialized_python_classB\x0b\n\t_sql_type\x1a\x34\n\x08Unparsed\x12(\n\x10\x64\x61ta_type_string\x18\x01 \x01(\tR\x0e\x64\x61taTypeStringB\x06\n\x04kindB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3" ) _globals = globals() @@ -47,59 +47,63 @@ "DESCRIPTOR" ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_DATATYPE"]._serialized_start = 45 - _globals["_DATATYPE"]._serialized_end = 4569 - _globals["_DATATYPE_BOOLEAN"]._serialized_start = 1647 - _globals["_DATATYPE_BOOLEAN"]._serialized_end = 1714 - _globals["_DATATYPE_BYTE"]._serialized_start = 1716 - _globals["_DATATYPE_BYTE"]._serialized_end = 1780 - _globals["_DATATYPE_SHORT"]._serialized_start = 1782 - _globals["_DATATYPE_SHORT"]._serialized_end = 1847 - _globals["_DATATYPE_INTEGER"]._serialized_start = 1849 - _globals["_DATATYPE_INTEGER"]._serialized_end = 1916 - _globals["_DATATYPE_LONG"]._serialized_start = 1918 - _globals["_DATATYPE_LONG"]._serialized_end = 1982 - _globals["_DATATYPE_FLOAT"]._serialized_start = 1984 - _globals["_DATATYPE_FLOAT"]._serialized_end = 2049 - _globals["_DATATYPE_DOUBLE"]._serialized_start = 2051 - _globals["_DATATYPE_DOUBLE"]._serialized_end = 2117 - _globals["_DATATYPE_STRING"]._serialized_start = 2119 - _globals["_DATATYPE_STRING"]._serialized_end = 2215 - _globals["_DATATYPE_BINARY"]._serialized_start = 2217 - _globals["_DATATYPE_BINARY"]._serialized_end = 2283 - _globals["_DATATYPE_NULL"]._serialized_start = 2285 - _globals["_DATATYPE_NULL"]._serialized_end = 2349 - _globals["_DATATYPE_TIMESTAMP"]._serialized_start = 2351 - _globals["_DATATYPE_TIMESTAMP"]._serialized_end = 2420 - _globals["_DATATYPE_DATE"]._serialized_start = 2422 - _globals["_DATATYPE_DATE"]._serialized_end = 2486 - _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_start = 2488 - _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_end = 2560 - _globals["_DATATYPE_TIME"]._serialized_start = 2562 - _globals["_DATATYPE_TIME"]._serialized_end = 2675 - _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_start = 2677 - _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_end = 2753 - _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_start = 2756 - _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_end = 2935 - _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_start = 2938 - _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_end = 3115 - _globals["_DATATYPE_CHAR"]._serialized_start = 3117 - _globals["_DATATYPE_CHAR"]._serialized_end = 3205 - _globals["_DATATYPE_VARCHAR"]._serialized_start = 3207 - _globals["_DATATYPE_VARCHAR"]._serialized_end = 3298 - _globals["_DATATYPE_DECIMAL"]._serialized_start = 3301 - _globals["_DATATYPE_DECIMAL"]._serialized_end = 3454 - _globals["_DATATYPE_STRUCTFIELD"]._serialized_start = 3457 - _globals["_DATATYPE_STRUCTFIELD"]._serialized_end = 3618 - _globals["_DATATYPE_STRUCT"]._serialized_start = 3620 - _globals["_DATATYPE_STRUCT"]._serialized_end = 3747 - _globals["_DATATYPE_ARRAY"]._serialized_start = 3750 - _globals["_DATATYPE_ARRAY"]._serialized_end = 3912 - _globals["_DATATYPE_MAP"]._serialized_start = 3915 - _globals["_DATATYPE_MAP"]._serialized_end = 4134 - _globals["_DATATYPE_VARIANT"]._serialized_start = 4136 - _globals["_DATATYPE_VARIANT"]._serialized_end = 4203 - _globals["_DATATYPE_UDT"]._serialized_start = 4206 - _globals["_DATATYPE_UDT"]._serialized_end = 4495 - _globals["_DATATYPE_UNPARSED"]._serialized_start = 4497 - _globals["_DATATYPE_UNPARSED"]._serialized_end = 4549 + _globals["_DATATYPE"]._serialized_end = 4869 + _globals["_DATATYPE_BOOLEAN"]._serialized_start = 1778 + _globals["_DATATYPE_BOOLEAN"]._serialized_end = 1845 + _globals["_DATATYPE_BYTE"]._serialized_start = 1847 + _globals["_DATATYPE_BYTE"]._serialized_end = 1911 + _globals["_DATATYPE_SHORT"]._serialized_start = 1913 + _globals["_DATATYPE_SHORT"]._serialized_end = 1978 + _globals["_DATATYPE_INTEGER"]._serialized_start = 1980 + _globals["_DATATYPE_INTEGER"]._serialized_end = 2047 + _globals["_DATATYPE_LONG"]._serialized_start = 2049 + _globals["_DATATYPE_LONG"]._serialized_end = 2113 + _globals["_DATATYPE_FLOAT"]._serialized_start = 2115 + _globals["_DATATYPE_FLOAT"]._serialized_end = 2180 + _globals["_DATATYPE_DOUBLE"]._serialized_start = 2182 + _globals["_DATATYPE_DOUBLE"]._serialized_end = 2248 + _globals["_DATATYPE_STRING"]._serialized_start = 2250 + _globals["_DATATYPE_STRING"]._serialized_end = 2346 + _globals["_DATATYPE_BINARY"]._serialized_start = 2348 + _globals["_DATATYPE_BINARY"]._serialized_end = 2414 + _globals["_DATATYPE_NULL"]._serialized_start = 2416 + _globals["_DATATYPE_NULL"]._serialized_end = 2480 + _globals["_DATATYPE_TIMESTAMP"]._serialized_start = 2482 + _globals["_DATATYPE_TIMESTAMP"]._serialized_end = 2551 + _globals["_DATATYPE_DATE"]._serialized_start = 2553 + _globals["_DATATYPE_DATE"]._serialized_end = 2617 + _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_start = 2619 + _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_end = 2691 + _globals["_DATATYPE_TIME"]._serialized_start = 2693 + _globals["_DATATYPE_TIME"]._serialized_end = 2806 + _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_start = 2808 + _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_end = 2884 + _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_start = 2887 + _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_end = 3066 + _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_start = 3069 + _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_end = 3246 + _globals["_DATATYPE_CHAR"]._serialized_start = 3248 + _globals["_DATATYPE_CHAR"]._serialized_end = 3336 + _globals["_DATATYPE_VARCHAR"]._serialized_start = 3338 + _globals["_DATATYPE_VARCHAR"]._serialized_end = 3429 + _globals["_DATATYPE_DECIMAL"]._serialized_start = 3432 + _globals["_DATATYPE_DECIMAL"]._serialized_end = 3585 + _globals["_DATATYPE_STRUCTFIELD"]._serialized_start = 3588 + _globals["_DATATYPE_STRUCTFIELD"]._serialized_end = 3749 + _globals["_DATATYPE_STRUCT"]._serialized_start = 3751 + _globals["_DATATYPE_STRUCT"]._serialized_end = 3878 + _globals["_DATATYPE_ARRAY"]._serialized_start = 3881 + _globals["_DATATYPE_ARRAY"]._serialized_end = 4043 + _globals["_DATATYPE_MAP"]._serialized_start = 4046 + _globals["_DATATYPE_MAP"]._serialized_end = 4265 + _globals["_DATATYPE_GEOMETRY"]._serialized_start = 4267 + _globals["_DATATYPE_GEOMETRY"]._serialized_end = 4355 + _globals["_DATATYPE_GEOGRAPHY"]._serialized_start = 4357 + _globals["_DATATYPE_GEOGRAPHY"]._serialized_end = 4446 + _globals["_DATATYPE_VARIANT"]._serialized_start = 4448 + _globals["_DATATYPE_VARIANT"]._serialized_end = 4515 + _globals["_DATATYPE_UDT"]._serialized_start = 4518 + _globals["_DATATYPE_UDT"]._serialized_end = 4807 + _globals["_DATATYPE_UNPARSED"]._serialized_start = 4809 + _globals["_DATATYPE_UNPARSED"]._serialized_end = 4861 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi index d46770c4f888e..3f625890a809b 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.pyi +++ b/python/pyspark/sql/connect/proto/types_pb2.pyi @@ -674,6 +674,46 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + class Geometry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SRID_FIELD_NUMBER: builtins.int + TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int + srid: builtins.int + type_variation_reference: builtins.int + def __init__( + self, + *, + srid: builtins.int = ..., + type_variation_reference: builtins.int = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "srid", b"srid", "type_variation_reference", b"type_variation_reference" + ], + ) -> None: ... + + class Geography(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SRID_FIELD_NUMBER: builtins.int + TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int + srid: builtins.int + type_variation_reference: builtins.int + def __init__( + self, + *, + srid: builtins.int = ..., + type_variation_reference: builtins.int = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "srid", b"srid", "type_variation_reference", b"type_variation_reference" + ], + ) -> None: ... + class Variant(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -821,6 +861,8 @@ class DataType(google.protobuf.message.Message): MAP_FIELD_NUMBER: builtins.int VARIANT_FIELD_NUMBER: builtins.int UDT_FIELD_NUMBER: builtins.int + GEOMETRY_FIELD_NUMBER: builtins.int + GEOGRAPHY_FIELD_NUMBER: builtins.int UNPARSED_FIELD_NUMBER: builtins.int TIME_FIELD_NUMBER: builtins.int @property @@ -878,6 +920,11 @@ class DataType(google.protobuf.message.Message): def udt(self) -> global___DataType.UDT: """UserDefinedType""" @property + def geometry(self) -> global___DataType.Geometry: + """Geospatial types""" + @property + def geography(self) -> global___DataType.Geography: ... + @property def unparsed(self) -> global___DataType.Unparsed: """UnparsedDataType""" @property @@ -909,6 +956,8 @@ class DataType(google.protobuf.message.Message): map: global___DataType.Map | None = ..., variant: global___DataType.Variant | None = ..., udt: global___DataType.UDT | None = ..., + geometry: global___DataType.Geometry | None = ..., + geography: global___DataType.Geography | None = ..., unparsed: global___DataType.Unparsed | None = ..., time: global___DataType.Time | None = ..., ) -> None: ... @@ -937,6 +986,10 @@ class DataType(google.protobuf.message.Message): b"double", "float", b"float", + "geography", + b"geography", + "geometry", + b"geometry", "integer", b"integer", "kind", @@ -996,6 +1049,10 @@ class DataType(google.protobuf.message.Message): b"double", "float", b"float", + "geography", + b"geography", + "geometry", + b"geometry", "integer", b"integer", "kind", @@ -1058,6 +1115,8 @@ class DataType(google.protobuf.message.Message): "map", "variant", "udt", + "geometry", + "geography", "unparsed", "time", ] diff --git a/sql/connect/common/src/main/protobuf/spark/connect/types.proto b/sql/connect/common/src/main/protobuf/spark/connect/types.proto index 1800e3885774f..caaa2340f95dd 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/types.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/types.proto @@ -67,15 +67,17 @@ message DataType { // UserDefinedType UDT udt = 23; + // Geospatial types + Geometry geometry = 26; + + Geography geography = 27; + // UnparsedDataType Unparsed unparsed = 24; Time time = 28; } - // Reserved for geometry and geography types - reserved 26, 27; - message Boolean { uint32 type_variation_reference = 1; } @@ -192,6 +194,16 @@ message DataType { uint32 type_variation_reference = 4; } + message Geometry { + int32 srid = 1; + uint32 type_variation_reference = 2; + } + + message Geography { + int32 srid = 1; + uint32 type_variation_reference = 2; + } + message Variant { uint32 type_variation_reference = 1; } From 3c1917f85af7fbe29ead87a20b769879674c92e8 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 4 Nov 2025 17:50:49 -0800 Subject: [PATCH 035/400] [SPARK-54177][BUILD] Upgrade gRPC to 1.76 and protobuf to 6.33 Second attempt of https://github.com/apache/spark/pull/52874, fixed the grpcio deps version for Python 3.14, which was missed in the previous attempt. ### What changes were proposed in this pull request? Bump gRPC from 1.67 to 1.76, with additional Python package upgrades for consistency: - `googleapis-common-protos==1.71.0` - `protobuf==6.33.0` And `buf v33.0` Fix the shading leaks of the `spark-connect` jar before ``` $ jar tf spark-connect_2.13-4.1.0-preview3.jar | grep '.class$' | grep -v 'org/apache/spark' | grep -v 'org/sparkproject' | grep -v 'META-INF' javax/annotation/Generated.class ... javax/ejb/EJB.class ... javax/persistence/PersistenceContext.class ... javax/xml/ws/WebServiceRef.class ... com/google/shopping/type/Price$Builder.class ... com/google/apps/card/v1/Widget$DataCase.class ... ``` after ``` $ jar tf spark-connect_2.13-4.2.0-SNAPSHOT.jar | grep '.class$' | grep -v 'org/apache/spark' | grep -v 'org/sparkproject' | grep -v 'META-INF' ``` ### Why are the changes needed? For Python: - [grpcio v1.75.1](https://github.com/grpc/grpc/releases/tag/v1.75.1) addes official Python 3.14 support - googleapis-common-proto v1.71.0 addes official Python 3.14 support, see https://github.com/googleapis/google-cloud-python/pull/14699 For Java: - v1.74 removes dependency on Tomcat's annotation API, see https://github.com/grpc/grpc-java/issues/9179 Check full release notes at: https://github.com/grpc/grpc/releases ### Does this PR introduce _any_ user-facing change? Maybe, reduce the potential conflict risks between Spark and user classes. ### How was this patch tested? Pass GHA, plus manual checks (see above sections). ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52879 from pan3793/SPARK-54177-2. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 83e49b7aca35f561b0015f17fd8ec865670a182b) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 4 +-- .github/workflows/build_python_connect.yml | 2 +- .github/workflows/maven_test.yml | 2 +- .github/workflows/pages.yml | 2 +- .../workflows/python_hosted_runner_test.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 4 +-- dev/infra/Dockerfile | 4 +-- dev/requirements.txt | 8 +++--- dev/spark-test-image/docs/Dockerfile | 2 +- dev/spark-test-image/lint/Dockerfile | 4 +-- dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- .../python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- .../python-minimum/Dockerfile | 2 +- .../python-ps-minimum/Dockerfile | 2 +- pom.xml | 3 +-- project/SparkBuild.scala | 4 +-- .../docs/source/getting_started/install.rst | 12 ++++----- python/packaging/classic/setup.py | 4 +-- python/packaging/client/setup.py | 4 +-- python/packaging/connect/setup.py | 4 +-- python/pyspark/sql/connect/proto/base_pb2.py | 4 +-- .../pyspark/sql/connect/proto/catalog_pb2.py | 4 +-- .../pyspark/sql/connect/proto/commands_pb2.py | 4 +-- .../pyspark/sql/connect/proto/common_pb2.py | 4 +-- .../sql/connect/proto/example_plugins_pb2.py | 4 +-- .../sql/connect/proto/expressions_pb2.py | 4 +-- .../sql/connect/proto/ml_common_pb2.py | 4 +-- python/pyspark/sql/connect/proto/ml_pb2.py | 4 +-- .../sql/connect/proto/pipelines_pb2.py | 4 +-- .../sql/connect/proto/relations_pb2.py | 4 +-- python/pyspark/sql/connect/proto/types_pb2.py | 4 +-- .../sql/streaming/proto/StateMessage_pb2.py | 8 +++--- sql/connect/client/jdbc/pom.xml | 4 --- sql/connect/client/jvm/pom.xml | 4 --- sql/connect/common/pom.xml | 26 ------------------- sql/connect/common/src/main/buf.gen.yaml | 14 +++++----- sql/connect/server/pom.xml | 14 +++++----- sql/core/src/main/buf.gen.yaml | 2 +- 43 files changed, 83 insertions(+), 116 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 3ba71108f5533..2609f84d98cc3 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -361,7 +361,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') || contains(matrix.modules, 'yarn') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' python3.11 -m pip list # Run the tests. - name: Run tests @@ -765,7 +765,7 @@ jobs: python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.11 -m pip install 'black==23.12.1' 'protobuf==5.29.5' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip install 'black==23.12.1' 'protobuf==6.33.0' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' python3.11 -m pip list - name: Python CodeGen check for branch-3.5 if: inputs.branch == 'branch-3.5' diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index cec37af22dd7b..b1ebb45b9cbc1 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -72,7 +72,7 @@ jobs: python packaging/client/setup.py sdist cd dist pip install pyspark*client-*.tar.gz - pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting + pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting - name: List Python packages run: python -m pip list - name: Run tests (local) diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 95c9aac33fc6c..7bbfc420e02ab 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -175,7 +175,7 @@ jobs: - name: Install Python packages (Python 3.11) if: contains(matrix.modules, 'resource-managers#yarn') || (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' python3.11 -m pip list # Run the tests using script command. # BSD's script command doesn't support -c option, and the usage is different from Linux's one. diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index e800b40106ee3..2bba3dcaf176d 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -63,7 +63,7 @@ jobs: pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' - name: Install Ruby for documentation generation uses: ruby/setup-ruby@v1 diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index 77e85222c29dd..d55eb1d93799b 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -148,7 +148,7 @@ jobs: python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ - python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ + python${{matrix.python}} -m pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge - name: List Python packages run: python${{matrix.python}} -m pip list diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 86be7e0a82295..679998b893927 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -94,7 +94,7 @@ ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 @@ -111,7 +111,7 @@ RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PI RUN python3.10 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ -'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ +'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' RUN python3.10 -m pip list diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 1aa03735ce926..423b6ba820d07 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -97,7 +97,7 @@ RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.3.3' scipy coverage matp ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 @@ -149,7 +149,7 @@ RUN apt-get update && apt-get install -y \ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 # TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ +RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.76.0 grpcio-status==1.76.0 lxml jinja2 && \ python3.13 -m pip cache purge # Remove unused installation packages to free up disk space diff --git a/dev/requirements.txt b/dev/requirements.txt index 76652df744815..ddaeb9b3dd9db 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -61,10 +61,10 @@ black==23.12.1 py # Spark Connect (required) -grpcio>=1.67.0 -grpcio-status>=1.67.0 -googleapis-common-protos>=1.65.0 -protobuf==5.29.5 +grpcio>=1.76.0 +grpcio-status>=1.76.0 +googleapis-common-protos>=1.71.0 +protobuf==6.33.0 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 1c17ae122d638..e268ea7a8351b 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -91,6 +91,6 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 RUN python3.11 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.5' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ && python3.11 -m pip cache purge diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 07ff9c90b7591..6686e3808e035 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -82,8 +82,8 @@ RUN python3.11 -m pip install \ 'flake8==3.9.0' \ 'googleapis-common-protos-stubs==2.2.0' \ 'grpc-stubs==1.24.11' \ - 'grpcio-status==1.67.0' \ - 'grpcio==1.67.0' \ + 'grpcio-status==1.76.0' \ + 'grpcio==1.76.0' \ 'ipython' \ 'ipython_genutils' \ 'jinja2' \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index d0409e61a51a6..d33fb5f5d30e4 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=21.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index ce2ca23d18a64..46cfce36832bf 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 00fb7be788fd0..920f21bd47ee7 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 79cab824a5b21..db1039f5cb260 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index 031eb8772b59b..a50bf670b3f35 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.13 packages diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index abd5a7e01093d..f74c48bf346f8 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 0ba9b620bd8b0..2a36da897e2df 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index 122281ec0ea1d..ebafbc69ec4d1 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" # Install Python 3.9 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 680697c3f2d74..13a5f2db386c8 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/pom.xml b/pom.xml index 107def928cbbb..b428823b3cb51 100644 --- a/pom.xml +++ b/pom.xml @@ -305,9 +305,8 @@ 33.4.0-jre 1.0.2 - 1.67.1 + 1.76.0 1.1.4 - 6.0.53 4.0-10 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1d8de063133e3..253893cc225b5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -820,11 +820,13 @@ object SparkConnect { ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.checkerframework.@1").inAll, ShadeRule.rename("com.google.gson.**" -> "org.sparkproject.connect.gson.@1").inAll, ShadeRule.rename("com.google.api.**" -> "org.sparkproject.connect.google_protos.api.@1").inAll, + ShadeRule.rename("com.google.apps.**" -> "org.sparkproject.connect.google_protos.apps.@1").inAll, ShadeRule.rename("com.google.cloud.**" -> "org.sparkproject.connect.google_protos.cloud.@1").inAll, ShadeRule.rename("com.google.geo.**" -> "org.sparkproject.connect.google_protos.geo.@1").inAll, ShadeRule.rename("com.google.logging.**" -> "org.sparkproject.connect.google_protos.logging.@1").inAll, ShadeRule.rename("com.google.longrunning.**" -> "org.sparkproject.connect.google_protos.longrunning.@1").inAll, ShadeRule.rename("com.google.rpc.**" -> "org.sparkproject.connect.google_protos.rpc.@1").inAll, + ShadeRule.rename("com.google.shopping.**" -> "org.sparkproject.connect.google_protos.shopping.@1").inAll, ShadeRule.rename("com.google.type.**" -> "org.sparkproject.connect.google_protos.type.@1").inAll ), @@ -911,7 +913,6 @@ object SparkConnectJdbc { ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, - ShadeRule.rename("javax.annotation.**" -> "org.sparkproject.connect.client.javax.annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll @@ -991,7 +992,6 @@ object SparkConnectClient { ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, - ShadeRule.rename("javax.annotation.**" -> "org.sparkproject.connect.client.javax.annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index 82db489651ff9..8b3c969d756d9 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -227,9 +227,9 @@ Package Supported version Note ========================== ================= ========================== `pandas` >=2.2.0 Required for Spark Connect `pyarrow` >=15.0.0 Required for Spark Connect -`grpcio` >=1.67.0 Required for Spark Connect -`grpcio-status` >=1.67.0 Required for Spark Connect -`googleapis-common-protos` >=1.65.0 Required for Spark Connect +`grpcio` >=1.76.0 Required for Spark Connect +`grpcio-status` >=1.76.0 Required for Spark Connect +`googleapis-common-protos` >=1.71.0 Required for Spark Connect `graphviz` >=0.20 Optional for Spark Connect ========================== ================= ========================== @@ -310,9 +310,9 @@ Package Supported version Note ========================== ================= =================================================== `pandas` >=2.2.0 Required for Spark Connect and Spark SQL `pyarrow` >=15.0.0 Required for Spark Connect and Spark SQL -`grpcio` >=1.67.0 Required for Spark Connect -`grpcio-status` >=1.67.0 Required for Spark Connect -`googleapis-common-protos` >=1.65.0 Required for Spark Connect +`grpcio` >=1.76.0 Required for Spark Connect +`grpcio-status` >=1.76.0 Required for Spark Connect +`googleapis-common-protos` >=1.71.0 Required for Spark Connect `pyyaml` >=3.11 Required for spark-pipelines command line interface `graphviz` >=0.20 Optional for Spark Connect ========================== ================= =================================================== diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index eac97af2e8c89..e6ac729f20d6f 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -153,8 +153,8 @@ def _supports_symlinks(): _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "15.0.0" -_minimum_grpc_version = "1.67.0" -_minimum_googleapis_common_protos_version = "1.65.0" +_minimum_grpc_version = "1.76.0" +_minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index 7ec7e45a31604..c378d223cfcc3 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -136,8 +136,8 @@ _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "15.0.0" - _minimum_grpc_version = "1.67.0" - _minimum_googleapis_common_protos_version = "1.65.0" + _minimum_grpc_version = "1.76.0" + _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" with open("README.md") as f: diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index f2b53211b3a0d..3b88563bcfe7e 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -89,8 +89,8 @@ _minimum_pandas_version = "2.0.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "11.0.0" - _minimum_grpc_version = "1.67.0" - _minimum_googleapis_common_protos_version = "1.65.0" + _minimum_grpc_version = "1.76.0" + _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" with open("README.md") as f: diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 0fe992332de71..32bf6802df7bc 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/base.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/base.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/base.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.py b/python/pyspark/sql/connect/proto/catalog_pb2.py index 58c129a01daa8..054b367bd3b34 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.py +++ b/python/pyspark/sql/connect/proto/catalog_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/catalog.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/catalog.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/catalog.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index 694b4a9a9aa37..4eccf1b71706d 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/commands.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/commands.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/commands.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/common_pb2.py b/python/pyspark/sql/connect/proto/common_pb2.py index 07ea9f7ed3173..8abd8fa6dc041 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.py +++ b/python/pyspark/sql/connect/proto/common_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/common.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/common.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/common.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.py b/python/pyspark/sql/connect/proto/example_plugins_pb2.py index 71a73a6d592ae..423768ee63d65 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.py +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/example_plugins.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/example_plugins.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/example_plugins.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index bd75ade02d8be..0c466aeb67a0d 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/expressions.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/expressions.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/expressions.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.py b/python/pyspark/sql/connect/proto/ml_common_pb2.py index a49491b8ad1ed..de547fc2a102f 100644 --- a/python/pyspark/sql/connect/proto/ml_common_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/ml_common.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/ml_common.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/ml_common.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py index 9574966472a58..3bd141815c8eb 100644 --- a/python/pyspark/sql/connect/proto/ml_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/ml.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/ml.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/ml.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py index f3489f55ed874..d7321fa7cf0c1 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.py +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/pipelines.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/pipelines.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/pipelines.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index e7f319554c5e2..9e630b6ba5e4c 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/relations.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/relations.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/relations.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/connect/proto/types_pb2.py b/python/pyspark/sql/connect/proto/types_pb2.py index 4e35f6b8911a2..fc5b14d068a87 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.py +++ b/python/pyspark/sql/connect/proto/types_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: spark/connect/types.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -27,7 +27,7 @@ from google.protobuf.internal import builder as _builder _runtime_version.ValidateProtobufRuntimeVersion( - _runtime_version.Domain.PUBLIC, 5, 29, 5, "", "spark/connect/types.proto" + _runtime_version.Domain.PUBLIC, 6, 33, 0, "", "spark/connect/types.proto" ) # @@protoc_insertion_point(imports) diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py index b88fc2c5ca402..1305a6213c137 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py @@ -18,7 +18,7 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! # NO CHECKED-IN PROTOBUF GENCODE # source: org/apache/spark/sql/execution/streaming/StateMessage.proto -# Protobuf Python Version: 5.29.5 +# Protobuf Python Version: 6.33.0 """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor from google.protobuf import descriptor_pool as _descriptor_pool @@ -28,9 +28,9 @@ _runtime_version.ValidateProtobufRuntimeVersion( _runtime_version.Domain.PUBLIC, - 5, - 29, - 5, + 6, + 33, + 0, "", "org/apache/spark/sql/execution/streaming/StateMessage.proto", ) diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index c2dda12b1e639..15e3480f2e59d 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -181,10 +181,6 @@ org.checkerframework ${spark.shade.packageName}.org.checkerframework - - javax.annotation - ${spark.shade.packageName}.javax.annotation - io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 1c16b7e9ca8ca..c5e4e6a1adfda 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -225,10 +225,6 @@ org.checkerframework ${spark.shade.packageName}.org.checkerframework - - javax.annotation - ${spark.shade.packageName}.javax.annotation - io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 6ff47ec6d68ce..2713f23c51e6a 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -87,11 +87,6 @@ netty-transport-native-unix-common ${netty.version} - - org.apache.tomcat - annotations-api - ${tomcat.annotations.api.version} - - org.apache.tomcat - annotations-api - ${tomcat.annotations.api.version} - provided - org.scalacheck scalacheck_${scala.binary.version} @@ -376,6 +370,10 @@ com.google.api ${spark.shade.packageName}.connect.google_protos.api + + com.google.apps + ${spark.shade.packageName}.connect.google_protos.apps + com.google.cloud ${spark.shade.packageName}.connect.google_protos.cloud @@ -396,6 +394,10 @@ com.google.rpc ${spark.shade.packageName}.connect.google_protos.rpc + + com.google.shopping + ${spark.shade.packageName}.connect.google_protos.shopping + com.google.type ${spark.shade.packageName}.connect.google_protos.type diff --git a/sql/core/src/main/buf.gen.yaml b/sql/core/src/main/buf.gen.yaml index 01a34ed308444..5f87a840c6a49 100644 --- a/sql/core/src/main/buf.gen.yaml +++ b/sql/core/src/main/buf.gen.yaml @@ -17,7 +17,7 @@ version: v1 plugins: # Building the Python build and building the mypy interfaces. - - plugin: buf.build/protocolbuffers/python:v29.5 + - plugin: buf.build/protocolbuffers/python:v33.0 out: gen/proto/python - name: mypy out: gen/proto/python From 1df646d695f922535ad3bc95e90af3b7bb48a238 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Tue, 4 Nov 2025 20:04:02 -0800 Subject: [PATCH 036/400] [SPARK-54162][GEO][SQL] Allow casting from GeographyType to GeometryType ### What changes were proposed in this pull request? This PR allows casting `GEOGRAPHY` to `GEOMETRY` if they have the same SRID. ### Why are the changes needed? Enable explicit casting between geospatial types. ### Does this PR introduce _any_ user-facing change? Yes, casting `GEOGRAPHY` to `GEOMETRY` is now allowed. ### How was this patch tested? Added new unit tests: - `StUtilsSuite` - `CastSuiteBase` Added new e2e SQL tests: - `st-functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52839 from uros-db/geo-cast-geog_geom. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit 84a3fa07ed5f7281282287f830ab6deefae5ccb9) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/util/STUtils.java | 9 +++++ .../spark/sql/catalyst/expressions/Cast.scala | 26 +++++++++++++- .../catalyst/expressions/CastSuiteBase.scala | 34 +++++++++++++++++++ .../spark/sql/catalyst/util/StUtilsSuite.java | 10 ++++++ .../nonansi/st-functions.sql.out | 29 ++++++++++++++++ .../analyzer-results/st-functions.sql.out | 29 ++++++++++++++++ .../sql-tests/inputs/st-functions.sql | 5 +++ .../results/nonansi/st-functions.sql.out | 32 +++++++++++++++++ .../sql-tests/results/st-functions.sql.out | 32 +++++++++++++++++ 9 files changed, 205 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java index 9edeee26eb98a..9aed051e06398 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java @@ -48,6 +48,15 @@ static GeometryVal toPhysVal(Geometry g) { return g.getValue(); } + /** Geospatial type casting utility methods. */ + + // Cast geography to geometry. + public static GeometryVal geographyToGeometry(GeographyVal geographyVal) { + // Geographic SRID is always a valid SRID for geometry, so we don't need to check it. + // Also, all geographic coordinates are valid for geometry, so no need to check bounds. + return toPhysVal(Geometry.fromBytes(geographyVal.getBytes())); + } + /** Geospatial type encoder/decoder utilities. */ public static GeometryVal serializeGeomFromWKB(org.apache.spark.sql.types.Geometry geometry, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 5b76c7d225e11..974cdfe1b012f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.IntervalUtils.{dayTimeIntervalToByte, import org.apache.spark.sql.errors.{QueryErrorsBase, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{UTF8String, VariantVal} +import org.apache.spark.unsafe.types.{GeographyVal, UTF8String, VariantVal} import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper} import org.apache.spark.util.ArrayImplicits._ @@ -164,6 +164,10 @@ object Cast extends QueryErrorsBase { case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true + // Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed. + case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid => + true + case _ => false } @@ -290,6 +294,10 @@ object Cast extends QueryErrorsBase { case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true + // Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed. + case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid => + true + case _ => false } @@ -1139,6 +1147,12 @@ case class Cast( b => numeric.toFloat(b) } + // GeometryConverter + private[this] def castToGeometry(from: DataType): Any => Any = from match { + case _: GeographyType => + buildCast[GeographyVal](_, STUtils.geographyToGeometry) + } + private[this] def castArray(fromType: DataType, toType: DataType): Any => Any = { val elementCast = cast(fromType, toType) // TODO: Could be faster? @@ -1218,6 +1232,7 @@ case class Cast( case FloatType => castToFloat(from) case LongType => castToLong(from) case DoubleType => castToDouble(from) + case _: GeometryType => castToGeometry(from) case array: ArrayType => castArray(from.asInstanceOf[ArrayType].elementType, array.elementType) case map: MapType => castMap(from.asInstanceOf[MapType], map) @@ -1326,6 +1341,7 @@ case class Cast( case FloatType => castToFloatCode(from, ctx) case LongType => castToLongCode(from, ctx) case DoubleType => castToDoubleCode(from, ctx) + case _: GeometryType => castToGeometryCode(from) case array: ArrayType => castArrayCode(from.asInstanceOf[ArrayType].elementType, array.elementType, ctx) @@ -2172,6 +2188,14 @@ case class Cast( } } + private[this] def castToGeometryCode(from: DataType): CastFunction = { + from match { + case _: GeographyType => + (c, evPrim, _) => + code"$evPrim = org.apache.spark.sql.catalyst.util.STUtils.geographyToGeometry($c);" + } + } + private[this] def castArrayCode( fromType: DataType, toType: DataType, ctx: CodegenContext): CastFunction = { val elementCast = nullSafeCastFunction(fromType, toType, ctx) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index 7a87c86b63c04..bf28e2e7eeb79 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -1489,6 +1489,40 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } } + // The following tests are confirming the behavior of casting between geospatial types. + + test("Casting GeographyType to GeometryType") { + // Casting from GEOGRAPHY to GEOMETRY is only allowed if the SRIDs are the same. + + // Valid cast test cases. + val canAnsiCastTestCases: Seq[(DataType, DataType)] = Seq( + (GeographyType(4326), GeometryType(4326)), + (GeographyType("ANY"), GeometryType("ANY")) + ) + // Iterate over the test cases and verify casting. + canAnsiCastTestCases.foreach { case (fromType, toType) => + // Cast can be performed from `fromType` to `toType`. + assert(Cast.canCast(fromType, toType)) + assert(Cast.canAnsiCast(fromType, toType)) + } + + // Invalid cast test cases. + val cannotAnsiCastTestCases: Seq[(DataType, DataType)] = Seq( + (GeographyType(4326), GeometryType(0)), + (GeographyType(4326), GeometryType(3857)), + (GeographyType(4326), GeometryType("ANY")), + (GeographyType("ANY"), GeometryType(0)), + (GeographyType("ANY"), GeometryType(3857)), + (GeographyType("ANY"), GeometryType(4326)) + ) + // Iterate over the test cases and verify casting. + cannotAnsiCastTestCases.foreach { case (fromType, toType) => + // Cast cannot be performed from `fromType` to `toType`. + assert(!Cast.canCast(fromType, toType)) + assert(!Cast.canAnsiCast(fromType, toType)) + } + } + test("cast string to time") { checkEvaluation(cast(Literal.create("0:0:0"), TimeType()), 0L) checkEvaluation(cast(Literal.create(" 01:2:3.01 "), TimeType(2)), localTime(1, 2, 3, 10000)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java index 8ad4d4c36e45c..0e18f8bdf919a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java @@ -62,6 +62,16 @@ class STUtilsSuite { System.arraycopy(testWkb, 0, testGeometryBytes, sridLen, wkbLen); } + /** Geospatial type casting utility methods. */ + + @Test + void testGeographyToGeometry() { + GeographyVal geographyVal = GeographyVal.fromBytes(testGeographyBytes); + GeometryVal geometryVal = STUtils.geographyToGeometry(geographyVal); + assertNotNull(geometryVal); + assertArrayEquals(geographyVal.getBytes(), geometryVal.getBytes()); + } + /** Tests for ST expression utility methods. */ // ST_AsBinary diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out index fe2dda3f1967b..dfc0b6b89a84f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out @@ -66,6 +66,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result +-- !query analysis +Project [hex(st_asbinary(cast(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040) as geometry(4326)))) AS result#x] ++- OneRowRelation + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY))\"", + "srcType" : "\"GEOGRAPHY(4326)\"", + "targetType" : "\"GEOMETRY(ANY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out index fe2dda3f1967b..dfc0b6b89a84f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out @@ -66,6 +66,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result +-- !query analysis +Project [hex(st_asbinary(cast(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040) as geometry(4326)))) AS result#x] ++- OneRowRelation + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY))\"", + "srcType" : "\"GEOGRAPHY(4326)\"", + "targetType" : "\"GEOMETRY(ANY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql index dc688e4a89941..6785a5b5254b6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql @@ -13,6 +13,11 @@ INSERT INTO geodata VALUES SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS STRING) AS result; SELECT CAST(X'0101000000000000000000f03f0000000000000040' AS GEOMETRY(4326)) AS result; +-- Casting GEOGRAPHY to GEOMETRY is allowed only if SRIDs match. +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result; +-- Error handling: mismatched SRIDs. +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)) AS result; + ---- ST reader/writer expressions -- WKB (Well-Known Binary) round-trip tests for GEOGRAPHY and GEOMETRY types. diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out index e75d4ba419e22..f1e876600a09f 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out @@ -73,6 +73,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result +-- !query schema +struct +-- !query output +0101000000000000000000F03F0000000000000040 + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY))\"", + "srcType" : "\"GEOGRAPHY(4326)\"", + "targetType" : "\"GEOMETRY(ANY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out index e75d4ba419e22..f1e876600a09f 100644 --- a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out @@ -73,6 +73,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result +-- !query schema +struct +-- !query output +0101000000000000000000F03F0000000000000040 + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY))\"", + "srcType" : "\"GEOGRAPHY(4326)\"", + "targetType" : "\"GEOMETRY(ANY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema From 096e3a31cd054fea1b2dfcaff9732a8edc63dc72 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Nov 2025 20:56:11 -0800 Subject: [PATCH 037/400] [SPARK-54188][K8S] Improve `ExecutorPodsWatchSnapshotSource` to watch only active executors ### What changes were proposed in this pull request? This PR aims to improve `ExecutorPodsWatchSnapshotSource` to watch only active executors like `ExecutorPodsPollingSnapshotSource`. ### Why are the changes needed? `ExecutorPodsPollingSnapshotSource` has been monitoring with `.withoutLabel(SPARK_EXECUTOR_INACTIVE_LABEL, "true")` condition. We had better make `ExecutorPodsWatchSnapshotSource` behave consistently. https://github.com/apache/spark/blob/a8e35c407bc5340f83b35e5a2f0b0767c6baadb0/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala#L77-L81 ### Does this PR introduce _any_ user-facing change? No. This will reduce Apache Spark's operation overhead because we ignore inactive pods. ### How was this patch tested? Pass the CIs with the revised unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52888 from dongjoon-hyun/SPARK-54188. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit a2f0c8375ab1271e4920f018af998a884384ed16) Signed-off-by: Dongjoon Hyun --- .../cluster/k8s/ExecutorPodsWatchSnapshotSource.scala | 1 + .../cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala index 6953ed789f797..0d9f19ee11b71 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala @@ -64,6 +64,7 @@ class ExecutorPodsWatchSnapshotSource( .inNamespace(namespace) .withLabel(SPARK_APP_ID_LABEL, applicationId) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .withoutLabel(SPARK_EXECUTOR_INACTIVE_LABEL, "true") .watch(new ExecutorPodsWatcher()) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala index 61080268cde60..f830abc0d1298 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala @@ -50,6 +50,9 @@ class ExecutorPodsWatchSnapshotSourceSuite extends SparkFunSuite with BeforeAndA @Mock private var executorRoleLabeledPods: LABELED_PODS = _ + @Mock + private var executorRoleLabeledActivePods: LABELED_PODS = _ + @Mock private var watchConnection: Watch = _ @@ -66,7 +69,9 @@ class ExecutorPodsWatchSnapshotSourceSuite extends SparkFunSuite with BeforeAndA .thenReturn(appIdLabeledPods) when(appIdLabeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) .thenReturn(executorRoleLabeledPods) - when(executorRoleLabeledPods.watch(watch.capture())).thenReturn(watchConnection) + when(executorRoleLabeledPods.withoutLabel(SPARK_EXECUTOR_INACTIVE_LABEL, "true")) + .thenReturn(executorRoleLabeledActivePods) + when(executorRoleLabeledActivePods.watch(watch.capture())).thenReturn(watchConnection) } test("Watch events should be pushed to the snapshots store as snapshot updates.") { From f1a93d19ca63f2df316f1834a3ccd08934018ef3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Nov 2025 21:56:51 -0800 Subject: [PATCH 038/400] [SPARK-54184][K8S] Support `spark.kubernetes.executor.deletedExecutorsCacheTimeout` ### What changes were proposed in this pull request? This PR aims to support `spark.kubernetes.executor.deletedExecutorsCacheTimeout`. ### Why are the changes needed? To allow users to control the TTL for the deleted executors cache. Previously, it has a hard-coded value. In some very slow clusters, we need to remember longer than 3 minutes. https://github.com/apache/spark/blob/a8e35c407bc5340f83b35e5a2f0b0767c6baadb0/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala#L54-L57 ### Does this PR introduce _any_ user-facing change? No behavior change because the default value is the same. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52884 from dongjoon-hyun/SPARK-54184. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit ada19082cfe7f091fe9d74b6aee6144673a444ef) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/deploy/k8s/Config.scala | 9 +++++++++ .../cluster/k8s/ExecutorPodsLifecycleManager.scala | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index f4d708f30b43a..0ae9b4a302fb9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -716,6 +716,15 @@ private[spark] object Config extends Logging { .booleanConf .createWithDefault(true) + val KUBERNETES_DELETED_EXECUTORS_CACHE_TIMEOUT = + ConfigBuilder("spark.kubernetes.executor.deletedExecutorsCacheTimeout") + .internal() + .doc("Time-to-live (TTL) value for the cache for deleted executors") + .version("4.1.0") + .timeConf(TimeUnit.SECONDS) + .checkValue(_ >= 0, "deletedExecutorsCacheTimeout must be non-negative") + .createWithDefault(180) + val KUBERNETES_EXECUTOR_TERMINATION_GRACE_PERIOD_SECONDS = ConfigBuilder("spark.kubernetes.executor.terminationGracePeriodSeconds") .doc("Time to wait for graceful termination of executor pods.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index 35386aff4a80c..9aa99a6c5984e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -53,7 +53,7 @@ private[spark] class ExecutorPodsLifecycleManager( // bounds. private lazy val removedExecutorsCache = CacheBuilder.newBuilder() - .expireAfterWrite(3, TimeUnit.MINUTES) + .expireAfterWrite(conf.get(KUBERNETES_DELETED_EXECUTORS_CACHE_TIMEOUT), TimeUnit.SECONDS) .build[java.lang.Long, java.lang.Long]() private var lastFullSnapshotTs: Long = 0 From 04e9f1f38f74cde54c7c7ca1a0d81211a6c17372 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Nov 2025 23:20:15 -0800 Subject: [PATCH 039/400] [SPARK-54189][K8S] Remove `spark.kubernetes.pyspark.pythonVersion` ### What changes were proposed in this pull request? This PR aims to remove no-op `spark.kubernetes.pyspark.pythonVersion` configuration. ### Why are the changes needed? `spark.kubernetes.pyspark.pythonVersion` was deprecated at Apache Spark 3.1.0 when Python 2 was dropped. It automatically fails when we use other values than `3`. And, technically no-op. We can remove this safely. https://github.com/apache/spark/blob/84a3fa07ed5f7281282287f830ab6deefae5ccb9/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala#L598 ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52889 from dongjoon-hyun/SPARK-54189. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 8b1ee1caea8680c18fa283fb72269f08b7fc1795) Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 11 ----------- .../org/apache/spark/deploy/k8s/Config.scala | 18 +----------------- .../features/DriverCommandFeatureStep.scala | 10 ---------- 3 files changed, 1 insertion(+), 38 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 68cd9a78d0f3a..bdc2a1a156b0e 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1284,17 +1284,6 @@ See the [configuration page](configuration.html) for information on Spark config 2.4.0 - - spark.kubernetes.pyspark.pythonVersion - "3" - - This sets the major Python version of the docker image used to run the driver and executor containers. - It can be only "3". This configuration was deprecated from Spark 3.1.0, and is effectively no-op. - Users should set 'spark.pyspark.python' and 'spark.pyspark.driver.python' configurations or - 'PYSPARK_PYTHON' and 'PYSPARK_DRIVER_PYTHON' environment variables. - - 2.4.0 - spark.kubernetes.kerberos.krb5.path (none) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 0ae9b4a302fb9..fafff5046b9dc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{ConfigBuilder, PYSPARK_DRIVER_PYTHON, PYSPARK_PYTHON} +import org.apache.spark.internal.config.ConfigBuilder private[spark] object Config extends Logging { @@ -587,22 +587,6 @@ private[spark] object Config extends Logging { "Ensure that memory overhead is non-negative") .createWithDefault(0.1) - val PYSPARK_MAJOR_PYTHON_VERSION = - ConfigBuilder("spark.kubernetes.pyspark.pythonVersion") - .doc( - s"(Deprecated since Spark 3.1, please set '${PYSPARK_PYTHON.key}' and " + - s"'${PYSPARK_DRIVER_PYTHON.key}' configurations or $ENV_PYSPARK_PYTHON and " + - s"$ENV_PYSPARK_DRIVER_PYTHON environment variables instead.)") - .version("2.4.0") - .stringConf - .checkValue("3" == _, - "Python 2 was dropped from Spark 3.1, and only 3 is allowed in " + - "this configuration. Note that this configuration was deprecated in Spark 3.1. " + - s"Please set '${PYSPARK_PYTHON.key}' and '${PYSPARK_DRIVER_PYTHON.key}' " + - s"configurations or $ENV_PYSPARK_PYTHON and $ENV_PYSPARK_DRIVER_PYTHON environment " + - "variables instead.") - .createOptional - val KUBERNETES_KERBEROS_KRB5_FILE = ConfigBuilder("spark.kubernetes.kerberos.krb5.path") .doc("Specify the local location of the krb5.conf file to be mounted on the driver " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index f15f5bc566b4b..0574fa4868f30 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -25,7 +25,6 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, CONFIG3, CONFIG4, CONFIG5} import org.apache.spark.internal.config.{PYSPARK_DRIVER_PYTHON, PYSPARK_PYTHON} import org.apache.spark.launcher.SparkLauncher @@ -77,15 +76,6 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesDriverConf) private[spark] def environmentVariables: Map[String, String] = sys.env private def configureForPython(pod: SparkPod, res: String): SparkPod = { - if (conf.get(PYSPARK_MAJOR_PYTHON_VERSION).isDefined) { - logWarning( - log"${MDC(CONFIG, PYSPARK_MAJOR_PYTHON_VERSION.key)} was deprecated in Spark 3.1. " + - log"Please set '${MDC(CONFIG2, PYSPARK_PYTHON.key)}' and " + - log"'${MDC(CONFIG3, PYSPARK_DRIVER_PYTHON.key)}' " + - log"configurations or ${MDC(CONFIG4, ENV_PYSPARK_PYTHON)} and " + - log"${MDC(CONFIG5, ENV_PYSPARK_DRIVER_PYTHON)} environment variables instead.") - } - val pythonEnvs = { KubernetesUtils.buildEnvVars( Seq( From 71599034d78686dfd119213add68fd3f519c7a2f Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 5 Nov 2025 07:43:54 -0800 Subject: [PATCH 040/400] [SPARK-54014][CONNECT] Support max rows for SparkConnectStatement ### What changes were proposed in this pull request? This PR implements the two methods of the `java.sql.Statement` interface for `SparkConnectStatement` ``` /** * Retrieves the maximum number of rows that a * {code ResultSet} object produced by this * {code Statement} object can contain. If this limit is exceeded, * the excess rows are silently dropped. * * return the current maximum number of rows for a {code ResultSet} * object produced by this {code Statement} object; * zero means there is no limit * throws SQLException if a database access error occurs or * this method is called on a closed {code Statement} * see #setMaxRows */ int getMaxRows() throws SQLException; /** * Sets the limit for the maximum number of rows that any * {code ResultSet} object generated by this {code Statement} * object can contain to the given number. * If the limit is exceeded, the excess * rows are silently dropped. * * param max the new max rows limit; zero means there is no limit * throws SQLException if a database access error occurs, * this method is called on a closed {code Statement} * or the condition {code max >= 0} is not satisfied * see #getMaxRows */ void setMaxRows(int max) throws SQLException; ``` ### Why are the changes needed? Implement more JDBC APIs. ### Does this PR introduce _any_ user-facing change? No, it's new feature. ### How was this patch tested? New UTs are added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52742 from pan3793/SPARK-54014. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 07cab004409c81d43001f9a0e309f5630ff5ab4c) Signed-off-by: Dongjoon Hyun --- .../client/jdbc/SparkConnectStatement.scala | 19 +++++++-- .../jdbc/SparkConnectStatementSuite.scala | 39 +++++++++++++++++++ 2 files changed, 54 insertions(+), 4 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala index 8b60f309ef6d3..3df1ff65498df 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatement.scala @@ -26,6 +26,8 @@ class SparkConnectStatement(conn: SparkConnectConnection) extends Statement { private var operationId: String = _ private var resultSet: SparkConnectResultSet = _ + private var maxRows: Int = 0 + @volatile private var closed: Boolean = false override def isClosed: Boolean = closed @@ -86,7 +88,10 @@ class SparkConnectStatement(conn: SparkConnectConnection) extends Statement { operationId = null resultSet = null - val df = conn.spark.sql(sql) + var df = conn.spark.sql(sql) + if (maxRows > 0) { + df = df.limit(maxRows) + } val sparkResult = df.collectResult() operationId = sparkResult.operationId if (hasResultSet(sparkResult)) { @@ -111,11 +116,17 @@ class SparkConnectStatement(conn: SparkConnectConnection) extends Statement { override def getMaxRows: Int = { checkOpen() - 0 + this.maxRows } - override def setMaxRows(max: Int): Unit = - throw new SQLFeatureNotSupportedException + override def setMaxRows(max: Int): Unit = { + checkOpen() + + if (max < 0) { + throw new SQLException("The max rows must be zero or a positive integer.") + } + this.maxRows = max + } override def setEscapeProcessing(enable: Boolean): Unit = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala index 8e3b616372d89..fa9df3f1247f7 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectStatementSuite.scala @@ -77,4 +77,43 @@ class SparkConnectStatementSuite extends ConnectFunSuite with RemoteSparkSession } } } + + test("max rows from SparkConnectStatement") { + def verifyMaxRows( + expectedRows: Int, query: String)(stmt: Statement): Unit = { + Using(stmt.executeQuery(query)) { rs => + (0 until expectedRows).foreach { _ => + assert(rs.next()) + } + assert(!rs.next()) + } + } + + withStatement { stmt => + // by default, it has no max rows limitation + assert(stmt.getMaxRows === 0) + verifyMaxRows(10, "SELECT id FROM range(10)")(stmt) + + val se = intercept[SQLException] { + stmt.setMaxRows(-1) + } + assert(se.getMessage === "The max rows must be zero or a positive integer.") + + stmt.setMaxRows(5) + assert(stmt.getMaxRows === 5) + verifyMaxRows(5, "SELECT id FROM range(10)")(stmt) + + // set max rows for query that has LIMIT + stmt.setMaxRows(5) + assert(stmt.getMaxRows === 5) + verifyMaxRows(3, "SELECT id FROM range(10) LIMIT 3")(stmt) + verifyMaxRows(5, "SELECT id FROM range(10) LIMIT 8")(stmt) + + // set max rows for one statement won't affect others + withStatement { stmt2 => + assert(stmt2.getMaxRows === 0) + verifyMaxRows(10, "SELECT id FROM range(10)")(stmt2) + } + } + } } From 16757ec907505e0049a97acbcac1296befb88cd3 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 5 Nov 2025 08:20:29 -0800 Subject: [PATCH 041/400] [SPARK-54180][SQL] Override the toString of BinaryFileFormat ### What changes were proposed in this pull request? Overrides the toString of BinaryFileFormat, making it similar to other file formats like Json and CSV. Not much impact other than making it prettier in the query plan. ### Why are the changes needed? Consistency ### How was this patch tested? It's a vanity thing, doesn't need much testing ### Was this patch authored or co-authored using generative AI tooling? no Closes #52881 from brkyvz/bToString. Authored-by: Burak Yavuz Signed-off-by: Dongjoon Hyun (cherry picked from commit 1af7d0defd7b0a02a1aee07eac41162f35837f2d) Signed-off-by: Dongjoon Hyun --- .../sql/execution/datasources/binaryfile/BinaryFileFormat.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala index 57e0efb993fb7..19b67d0c53900 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala @@ -83,6 +83,8 @@ case class BinaryFileFormat() extends FileFormat override def shortName(): String = BINARY_FILE + override def toString: String = "BINARYFILE" + override protected def buildReader( sparkSession: SparkSession, dataSchema: StructType, From bde919479e0143b2f44cff4c1b7debc312e838d2 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Wed, 5 Nov 2025 10:49:50 -0800 Subject: [PATCH 042/400] [SPARK-54176][GEO][PYTHON] Introduce Geography and Geometry data types to PySpark Connect ### What changes were proposed in this pull request? Introduce `GeographyType` and `GeometryType` to PySpark Connect. Note that the geospatial data types have already been introduced in PySpark as part of: https://github.com/apache/spark/pull/52627. Also, introduce classes to represent a `Geography` and `Geometry` value in Python. Note that the corresponding classes have already been introduced on Scala side as part of: https://github.com/apache/spark/pull/52804. ### Why are the changes needed? Enabling geospatial types in Spark Connect. ### Does this PR introduce _any_ user-facing change? Yes, `GeographyType` and `GeometryType` are now available in PySpark Connect. ### How was this patch tested? Added new Python Connect tests: - `test_parity_geographytype` - `test_parity_geometrytype` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52871 from uros-db/geo-spark-connect. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan --- dev/sparktestsupport/modules.py | 2 + python/pyspark/errors/error-conditions.json | 10 ++ python/pyspark/sql/connect/types.py | 18 ++ python/pyspark/sql/conversion.py | 74 ++++++++ .../connect/test_parity_geographytype.py | 38 ++++ .../tests/connect/test_parity_geometrytype.py | 38 ++++ python/pyspark/sql/types.py | 168 ++++++++++++++++++ 7 files changed, 348 insertions(+) create mode 100644 python/pyspark/sql/tests/connect/test_parity_geographytype.py create mode 100644 python/pyspark/sql/tests/connect/test_parity_geometrytype.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 07ac4c76b91a6..aa8ca58a5a75f 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1114,6 +1114,8 @@ def __hash__(self): "pyspark.sql.tests.connect.test_connect_retry", "pyspark.sql.tests.connect.test_connect_session", "pyspark.sql.tests.connect.test_connect_stat", + "pyspark.sql.tests.connect.test_parity_geographytype", + "pyspark.sql.tests.connect.test_parity_geometrytype", "pyspark.sql.tests.connect.test_parity_datasources", "pyspark.sql.tests.connect.test_parity_errors", "pyspark.sql.tests.connect.test_parity_catalog", diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index d169e6293a1ba..51bbdd8625164 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -549,6 +549,16 @@ " and should be of the same length, got and ." ] }, + "MALFORMED_GEOGRAPHY": { + "message": [ + "Geography binary is malformed. Please check the data source is valid." + ] + }, + "MALFORMED_GEOMETRY": { + "message": [ + "Geometry binary is malformed. Please check the data source is valid." + ] + }, "MALFORMED_VARIANT": { "message": [ "Variant binary is malformed. Please check the data source is valid." diff --git a/python/pyspark/sql/connect/types.py b/python/pyspark/sql/connect/types.py index 7e8f768610794..d3352b618d7c7 100644 --- a/python/pyspark/sql/connect/types.py +++ b/python/pyspark/sql/connect/types.py @@ -50,6 +50,8 @@ NullType, NumericType, VariantType, + GeographyType, + GeometryType, UserDefinedType, ) from pyspark.errors import PySparkAssertionError, PySparkValueError @@ -191,6 +193,10 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType: ret.array.contains_null = data_type.containsNull elif isinstance(data_type, VariantType): ret.variant.CopyFrom(pb2.DataType.Variant()) + elif isinstance(data_type, GeometryType): + ret.geometry.srid = data_type.srid + elif isinstance(data_type, GeographyType): + ret.geography.srid = data_type.srid elif isinstance(data_type, UserDefinedType): json_value = data_type.jsonValue() ret.udt.type = "udt" @@ -303,6 +309,18 @@ def proto_schema_to_pyspark_data_type(schema: pb2.DataType) -> DataType: ) elif schema.HasField("variant"): return VariantType() + elif schema.HasField("geometry"): + srid = schema.geometry.srid + if srid == GeometryType.MIXED_SRID: + return GeometryType("ANY") + else: + return GeometryType(srid) + elif schema.HasField("geography"): + srid = schema.geography.srid + if srid == GeographyType.MIXED_SRID: + return GeographyType("ANY") + else: + return GeographyType(srid) elif schema.HasField("udt"): assert schema.udt.type == "udt" json_value = {} diff --git a/python/pyspark/sql/conversion.py b/python/pyspark/sql/conversion.py index a8f621277a0af..f73727d1d5344 100644 --- a/python/pyspark/sql/conversion.py +++ b/python/pyspark/sql/conversion.py @@ -28,6 +28,10 @@ BinaryType, DataType, DecimalType, + GeographyType, + Geography, + GeometryType, + Geometry, MapType, NullType, Row, @@ -89,6 +93,10 @@ def _need_converter( return True elif isinstance(dataType, VariantType): return True + elif isinstance(dataType, GeometryType): + return True + elif isinstance(dataType, GeographyType): + return True else: return False @@ -392,6 +400,34 @@ def convert_variant(value: Any) -> Any: return convert_variant + elif isinstance(dataType, GeographyType): + + def convert_geography(value: Any) -> Any: + if value is None: + if not nullable: + raise PySparkValueError(f"input for {dataType} must not be None") + return None + elif isinstance(value, Geography): + return dataType.toInternal(value) + else: + raise PySparkValueError(errorClass="MALFORMED_GEOGRAPHY") + + return convert_geography + + elif isinstance(dataType, GeometryType): + + def convert_geometry(value: Any) -> Any: + if value is None: + if not nullable: + raise PySparkValueError(f"input for {dataType} must not be None") + return None + elif isinstance(value, Geometry): + return dataType.toInternal(value) + else: + raise PySparkValueError(errorClass="MALFORMED_GEOMETRY") + + return convert_geometry + elif not nullable: def convert_other(value: Any) -> Any: @@ -511,6 +547,10 @@ def _need_converter(dataType: DataType) -> bool: return True elif isinstance(dataType, VariantType): return True + elif isinstance(dataType, GeographyType): + return True + elif isinstance(dataType, GeometryType): + return True else: return False @@ -719,6 +759,40 @@ def convert_variant(value: Any) -> Any: return convert_variant + elif isinstance(dataType, GeographyType): + + def convert_geography(value: Any) -> Any: + if value is None: + return None + elif ( + isinstance(value, dict) + and all(key in value for key in ["wkb", "srid"]) + and isinstance(value["wkb"], bytes) + and isinstance(value["srid"], int) + ): + return Geography.fromWKB(value["wkb"], value["srid"]) + else: + raise PySparkValueError(errorClass="MALFORMED_GEOGRAPHY") + + return convert_geography + + elif isinstance(dataType, GeometryType): + + def convert_geometry(value: Any) -> Any: + if value is None: + return None + elif ( + isinstance(value, dict) + and all(key in value for key in ["wkb", "srid"]) + and isinstance(value["wkb"], bytes) + and isinstance(value["srid"], int) + ): + return Geometry.fromWKB(value["wkb"], value["srid"]) + else: + raise PySparkValueError(errorClass="MALFORMED_GEOMETRY") + + return convert_geometry + else: if none_on_identity: return None diff --git a/python/pyspark/sql/tests/connect/test_parity_geographytype.py b/python/pyspark/sql/tests/connect/test_parity_geographytype.py new file mode 100644 index 0000000000000..501bbed20ff19 --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_geographytype.py @@ -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. +# + +import unittest + +from pyspark.sql.tests.test_geographytype import GeographyTypeTestMixin +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class GeographyTypeParityTest(GeographyTypeTestMixin, ReusedConnectTestCase): + pass + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.connect.test_parity_geographytype import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/connect/test_parity_geometrytype.py b/python/pyspark/sql/tests/connect/test_parity_geometrytype.py new file mode 100644 index 0000000000000..b95321b3c61be --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_geometrytype.py @@ -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. +# + +import unittest + +from pyspark.sql.tests.test_geometrytype import GeometryTypeTestMixin +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class GeometryTypeParityTest(GeometryTypeTestMixin, ReusedConnectTestCase): + pass + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.connect.test_parity_geometrytype import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 440100dba9312..8aae398800727 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -90,6 +90,8 @@ "TimestampNTZType", "DecimalType", "DoubleType", + "Geography", + "Geometry", "FloatType", "ByteType", "IntegerType", @@ -616,6 +618,20 @@ def jsonValue(self) -> Union[str, Dict[str, Any]]: # The JSON representation always uses the CRS and algorithm value. return f"geography({self._crs}, {self._alg})" + def needConversion(self) -> bool: + return True + + def fromInternal(self, obj: Dict) -> Optional["Geography"]: + if obj is None or not all(key in obj for key in ["srid", "bytes"]): + return None + return Geography(obj["bytes"], obj["srid"]) + + def toInternal(self, geography: Any) -> Any: + if geography is None: + return None + assert isinstance(geography, Geography) + return {"srid": geography.srid, "wkb": geography.wkb} + class GeometryType(SpatialType): """ @@ -700,6 +716,20 @@ def jsonValue(self) -> Union[str, Dict[str, Any]]: # The JSON representation always uses the CRS value. return f"geometry({self._crs})" + def needConversion(self) -> bool: + return True + + def fromInternal(self, obj: Dict) -> Optional["Geometry"]: + if obj is None or not all(key in obj for key in ["srid", "bytes"]): + return None + return Geometry(obj["bytes"], obj["srid"]) + + def toInternal(self, geometry: Any) -> Any: + if geometry is None: + return None + assert isinstance(geometry, Geometry) + return {"srid": geometry.srid, "wkb": geometry.wkb} + class ByteType(IntegralType): """Byte data type, representing signed 8-bit integers.""" @@ -2039,6 +2069,144 @@ def parseJson(cls, json_str: str) -> "VariantVal": return VariantVal(value, metadata) +class Geography: + """ + A class to represent a Geography value in Python. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + wkb : bytes + The bytes representing the WKB of Geography. + + srid : integer + The integer value representing SRID of Geography. + + Methods + ------- + getBytes() + Returns the WKB of Geography. + + getSrid() + Returns the SRID of Geography. + + Examples + -------- + >>> g = Geography.fromWKB(bytes.fromhex('010100000000000000000031400000000000001c40'), 4326) + >>> g.getBytes().hex() + '010100000000000000000031400000000000001c40' + >>> g.getSrid() + 4326 + """ + + def __init__(self, wkb: bytes, srid: int): + self.wkb = wkb + self.srid = srid + + def __str__(self) -> str: + return "Geography(%r, %d)" % (self.wkb, self.srid) + + def __repr__(self) -> str: + return "Geography(%r, %d)" % (self.wkb, self.srid) + + def getSrid(self) -> int: + """ + Returns the SRID of Geography. + """ + return self.srid + + def getBytes(self) -> bytes: + """ + Returns the WKB of Geography. + """ + return self.wkb + + def __eq__(self, other: Any) -> bool: + if not isinstance(other, Geography): + # Don't attempt to compare against unrelated types. + return NotImplemented + + return self.wkb == other.wkb and self.srid == other.srid + + @classmethod + def fromWKB(cls, wkb: bytes, srid: int) -> "Geography": + """ + Construct Python Geography object from WKB. + :return: Python representation of the Geography type value. + """ + return Geography(wkb, srid) + + +class Geometry: + """ + A class to represent a Geometry value in Python. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + wkb : bytes + The bytes representing the WKB of Geometry. + + srid : integer + The integer value representing SRID of Geometry. + + Methods + ------- + getBytes() + Returns the WKB of Geometry. + + getSrid() + Returns the SRID of Geometry. + + Examples + -------- + >>> g = Geometry.fromWKB(bytes.fromhex('010100000000000000000031400000000000001c40'), 0) + >>> g.getBytes().hex() + '010100000000000000000031400000000000001c40' + >>> g.getSrid() + 0 + """ + + def __init__(self, wkb: bytes, srid: int): + self.wkb = wkb + self.srid = srid + + def __str__(self) -> str: + return "Geometry(%r, %d)" % (self.wkb, self.srid) + + def __repr__(self) -> str: + return "Geometry(%r, %d)" % (self.wkb, self.srid) + + def getSrid(self) -> int: + """ + Returns the SRID of Geometry. + """ + return self.srid + + def getBytes(self) -> bytes: + """ + Returns the WKB of Geometry. + """ + return self.wkb + + def __eq__(self, other: Any) -> bool: + if not isinstance(other, Geometry): + # Don't attempt to compare against unrelated types. + return NotImplemented + + return self.wkb == other.wkb and self.srid == other.srid + + @classmethod + def fromWKB(cls, wkb: bytes, srid: int) -> "Geometry": + """ + Construct Python Geometry object from WKB. + :return: Python representation of the Geometry type value. + """ + return Geometry(wkb, srid) + + _atomic_types: List[Type[DataType]] = [ StringType, CharType, From 0e24148d1f8750c5ae3267f6101ff5807b925e7e Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Mon, 3 Nov 2025 10:16:20 +0800 Subject: [PATCH 043/400] [SPARK-54079][GEO][SQL][FOLLOWUP] Fix catalyst expressions ST package and imports ### What changes were proposed in this pull request? Following up on the original PR (https://github.com/apache/spark/pull/52784), this PR fixes catalyst expressions ST package and imports. ### Why are the changes needed? Use proper package and referencing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests suffice. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52807 from uros-db/geo-st-expressions-package. Authored-by: Uros Bojanic Signed-off-by: Ruifeng Zheng --- .../spark/sql/catalyst/analysis/FunctionRegistry.scala | 1 + .../spark/sql/catalyst/expressions/st/stExpressions.scala | 3 ++- .../test/resources/sql-functions/sql-expression-schema.md | 8 ++++---- .../scala/org/apache/spark/sql/STExpressionsSuite.scala | 1 + 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index bb98e5fa02ed2..3098832774d17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.st._ import org.apache.spark.sql.catalyst.expressions.variant._ import org.apache.spark.sql.catalyst.expressions.xml._ import org.apache.spark.sql.catalyst.plans.logical.{FunctionBuilderBase, Generate, LogicalPlan, OneRowRelation, Range} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala index a08f88b679525..135a7f7d61f21 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions +package org.apache.spark.sql.catalyst.expressions.st +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.util.{Geography, Geometry, STUtils} diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 29d194f5715e6..d9e4a960a2d78 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -287,10 +287,6 @@ | org.apache.spark.sql.catalyst.expressions.Rint | rint | SELECT rint(12.3456) | struct | | org.apache.spark.sql.catalyst.expressions.Round | round | SELECT round(2.5, 0) | struct | | org.apache.spark.sql.catalyst.expressions.RowNumber | row_number | SELECT a, b, row_number() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | -| org.apache.spark.sql.catalyst.expressions.ST_AsBinary | st_asbinary | SELECT hex(st_asbinary(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | -| org.apache.spark.sql.catalyst.expressions.ST_GeogFromWKB | st_geogfromwkb | SELECT hex(st_asbinary(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | -| org.apache.spark.sql.catalyst.expressions.ST_GeomFromWKB | st_geomfromwkb | SELECT hex(st_asbinary(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | -| org.apache.spark.sql.catalyst.expressions.ST_Srid | st_srid | SELECT st_srid(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040')) | struct | | org.apache.spark.sql.catalyst.expressions.SchemaOfCsv | schema_of_csv | SELECT schema_of_csv('1,abc') | struct | | org.apache.spark.sql.catalyst.expressions.SchemaOfJson | schema_of_json | SELECT schema_of_json('[{"col":0}]') | struct | | org.apache.spark.sql.catalyst.expressions.SchemaOfXml | schema_of_xml | SELECT schema_of_xml('

1

') | struct1

):string> | @@ -481,6 +477,10 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.VariancePop | var_pop | SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | var_samp | SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | variance | SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.st.ST_AsBinary | st_asbinary | SELECT hex(st_asbinary(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | +| org.apache.spark.sql.catalyst.expressions.st.ST_GeogFromWKB | st_geogfromwkb | SELECT hex(st_asbinary(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | +| org.apache.spark.sql.catalyst.expressions.st.ST_GeomFromWKB | st_geomfromwkb | SELECT hex(st_asbinary(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | +| org.apache.spark.sql.catalyst.expressions.st.ST_Srid | st_srid | SELECT st_srid(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040')) | struct | | org.apache.spark.sql.catalyst.expressions.variant.IsVariantNull | is_variant_null | SELECT is_variant_null(parse_json('null')) | struct | | org.apache.spark.sql.catalyst.expressions.variant.ParseJsonExpressionBuilder | parse_json | SELECT parse_json('{"a":1,"b":0.8}') | struct | | org.apache.spark.sql.catalyst.expressions.variant.SchemaOfVariant | schema_of_variant | SELECT schema_of_variant(parse_json('null')) | struct | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 71353e1bc29b2..1b73617d050e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.st._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ From 77d034d71e92b6810118ec06c0f3f2334c58a1c7 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Wed, 5 Nov 2025 13:00:06 -0800 Subject: [PATCH 044/400] [SPARK-54160][GEO][SQL] Implement the ST_SetSrid expression in SQL ### What changes were proposed in this pull request? Implement the `ST_SetSrid` expression in Catalyst, and add the corresponding `st_setsrid` SQL function. #### Description Returns a new `GEOGRAPHY` or `GEOMETRY` value whose `SRID` is the specified SRID value. #### Syntax ``` st_setsrid(geo, srid) ``` #### Arguments - `geo`: A `GEOGRAPHY` or `GEOMETRY` value. - `srid`: An `INTEGER` value. #### Returns - A value of type `GEOGRAPHY` or `GEOMETRY`. #### Notes - The function returns `NULL` if the input is `NULL`. - The function returns an `ST_INVALID_SRID_VALUE` error if the provided `srid` is not valid. #### Examples ``` > SELECT st_srid(st_setsrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)); 4326 ``` ``` > SELECT st_srid(st_setsrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)); 3857 ``` ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? Yes, a new SQL function is added: `st_setsrid`. ### How was this patch tested? New unit tests in Java/Scala suites: - `STExpressionsSuite.scala` - `StUtilsSuite.java` New end-to-end SQL tests: - `st-functions.sql` Finally, manually testing using Spark shell. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52858 from uros-db/geo-ST_SetSrid. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan --- .../spark/sql/types/GeographyType.scala | 2 +- .../apache/spark/sql/types/GeometryType.scala | 2 +- .../apache/spark/sql/catalyst/util/Geo.java | 3 + .../spark/sql/catalyst/util/Geography.java | 15 +- .../spark/sql/catalyst/util/Geometry.java | 15 +- .../spark/sql/catalyst/util/STUtils.java | 26 ++++ .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/st/STExpressionUtils.scala | 46 +++++++ .../expressions/st/stExpressions.scala | 53 ++++++++ .../sql/errors/QueryExecutionErrors.scala | 11 ++ .../spark/sql/catalyst/util/StUtilsSuite.java | 46 +++++++ .../sql-functions/sql-expression-schema.md | 1 + .../nonansi/st-functions.sql.out | 84 ++++++++++++ .../analyzer-results/st-functions.sql.out | 84 ++++++++++++ .../sql-tests/inputs/st-functions.sql | 18 +++ .../results/nonansi/st-functions.sql.out | 92 +++++++++++++ .../sql-tests/results/st-functions.sql.out | 92 +++++++++++++ .../apache/spark/sql/STExpressionsSuite.scala | 128 ++++++++++++++++++ 18 files changed, 715 insertions(+), 4 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala index d72e5987abebd..4d6ab7e9c8e50 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala @@ -179,7 +179,7 @@ object GeographyType extends SpatialType { GeographyType(MIXED_CRS, GEOGRAPHY_DEFAULT_ALGORITHM) /** Returns whether the given SRID is supported. */ - private[types] def isSridSupported(srid: Int): Boolean = { + def isSridSupported(srid: Int): Boolean = { GeographicSpatialReferenceSystemMapper.getStringId(srid) != null } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala index f5bbbcba6706e..ff4a720118407 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala @@ -171,7 +171,7 @@ object GeometryType extends SpatialType { GeometryType(MIXED_CRS) /** Returns whether the given SRID is supported. */ - private[types] def isSridSupported(srid: Int): Boolean = { + def isSridSupported(srid: Int): Boolean = { CartesianSpatialReferenceSystemMapper.getStringId(srid) != null } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geo.java index 2299f35988638..bf723a8efef91 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geo.java @@ -78,4 +78,7 @@ interface Geo { // Returns the Spatial Reference Identifier (SRID) value of the geo object. int srid(); + // Sets the Spatial Reference Identifier (SRID) value of the geo object. + void setSrid(int srid); + } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geography.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geography.java index c46c2368832fe..da513d399f8b0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geography.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geography.java @@ -162,7 +162,20 @@ public byte[] toEwkt() { @Override public int srid() { // This method gets the SRID value from the in-memory Geography representation header. - return ByteBuffer.wrap(getBytes()).order(DEFAULT_ENDIANNESS).getInt(SRID_OFFSET); + return getWrapper().getInt(SRID_OFFSET); + } + + @Override + public void setSrid(int srid) { + // This method sets the SRID value in the in-memory Geography representation header. + getWrapper().putInt(SRID_OFFSET, srid); + } + + /** Other private helper/utility methods used for implementation. */ + + // Returns a byte buffer wrapper over the byte buffer of this geography value. + private ByteBuffer getWrapper() { + return ByteBuffer.wrap(getBytes()).order(DEFAULT_ENDIANNESS); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geometry.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geometry.java index c4b6e5d0e4bd1..36fffef2abbd1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geometry.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/Geometry.java @@ -162,7 +162,20 @@ public byte[] toEwkt() { @Override public int srid() { // This method gets the SRID value from the in-memory Geometry representation header. - return ByteBuffer.wrap(getBytes()).order(DEFAULT_ENDIANNESS).getInt(SRID_OFFSET); + return getWrapper().getInt(SRID_OFFSET); + } + + @Override + public void setSrid(int srid) { + // This method sets the SRID value in the in-memory Geometry representation header. + getWrapper().putInt(SRID_OFFSET, srid); + } + + /** Other private helper/utility methods used for implementation. */ + + // Returns a byte buffer wrapper over the byte buffer of this geometry value. + private ByteBuffer getWrapper() { + return ByteBuffer.wrap(getBytes()).order(DEFAULT_ENDIANNESS); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java index 9aed051e06398..3cf4b84ac0330 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.catalyst.util; +import org.apache.spark.sql.errors.QueryExecutionErrors; import org.apache.spark.sql.types.GeographyType; import org.apache.spark.sql.types.GeometryType; import org.apache.spark.unsafe.types.GeographyVal; @@ -110,6 +111,31 @@ public static GeometryVal stGeomFromWKB(byte[] wkb) { return toPhysVal(Geometry.fromWkb(wkb)); } + // ST_SetSrid + public static GeographyVal stSetSrid(GeographyVal geo, int srid) { + // We only allow setting the SRID to geographic values. + if(!GeographyType.isSridSupported(srid)) { + throw QueryExecutionErrors.stInvalidSridValueError(srid); + } + // Create a copy of the input geography. + Geography copy = fromPhysVal(geo).copy(); + // Set the SRID of the copy to the specified value. + copy.setSrid(srid); + return toPhysVal(copy); + } + + public static GeometryVal stSetSrid(GeometryVal geo, int srid) { + // We only allow setting the SRID to valid values. + if(!GeometryType.isSridSupported(srid)) { + throw QueryExecutionErrors.stInvalidSridValueError(srid); + } + // Create a copy of the input geometry. + Geometry copy = fromPhysVal(geo).copy(); + // Set the SRID of the copy to the specified value. + copy.setSrid(srid); + return toPhysVal(copy); + } + // ST_Srid public static int stSrid(GeographyVal geog) { return fromPhysVal(geog).srid(); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 3098832774d17..8ded5873f14a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -879,6 +879,7 @@ object FunctionRegistry { expression[ST_GeogFromWKB]("st_geogfromwkb"), expression[ST_GeomFromWKB]("st_geomfromwkb"), expression[ST_Srid]("st_srid"), + expression[ST_SetSrid]("st_setsrid"), // cast expression[Cast]("cast"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala index 055173ec39ade..7b0345d277bc1 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.st +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ private[sql] object STExpressionUtils { @@ -29,4 +30,49 @@ private[sql] object STExpressionUtils { case _ => false } + /** + * Returns the input GEOMETRY or GEOGRAPHY value with the specified SRID. Only geospatial types + * are allowed as the source type, and calls are delegated to the corresponding helper methods. + */ + def geospatialTypeWithSrid(sourceType: DataType, srid: Expression): DataType = { + sourceType match { + case _: GeometryType => + geometryTypeWithSrid(srid) + case _: GeographyType => + geographyTypeWithSrid(srid) + case _ => + throw new IllegalArgumentException(s"Unexpected data type: $sourceType.") + } + } + + /** + * Returns the input GEOMETRY value with the specified SRID. If the SRID expression is a literal, + * the SRID value can be directly extracted. Otherwise, only the mixed SRID value can be used. + */ + private def geometryTypeWithSrid(srid: Expression): GeometryType = { + srid match { + case Literal(sridValue: Int, IntegerType) => + // If the SRID expression is a literal, the SRID value can be directly extracted. + GeometryType(sridValue) + case _ => + // Otherwise, only the mixed SRID value can be used for the output GEOMETRY value. + GeometryType("ANY") + } + } + + /** + * Returns the input GEOGRAPHY value with the specified SRID. If the SRID expression is a literal, + * the SRID value can be directly extracted. Otherwise, only the mixed SRID value can be used. + */ + private def geographyTypeWithSrid(srid: Expression): GeographyType = { + srid match { + case Literal(sridValue: Int, IntegerType) => + // If the SRID expression is a literal, the SRID value can be directly extracted. + GeographyType(sridValue) + case _ => + // Otherwise, only the mixed SRID value can be used for the output GEOMETRY value. + GeographyType("ANY") + } + } + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala index 135a7f7d61f21..4cdfbeead8753 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala @@ -223,3 +223,56 @@ case class ST_Srid(geo: Expression) override protected def withNewChildInternal(newChild: Expression): ST_Srid = copy(geo = newChild) } + +/** ST modifier expressions. */ + +/** + * Returns a new GEOGRAPHY or GEOMETRY value whose SRID is the specified SRID value. + */ +@ExpressionDescription( + usage = "_FUNC_(geo, srid) - Returns a new GEOGRAPHY or GEOMETRY value whose SRID is " + + "the specified SRID value.", + arguments = """ + Arguments: + * geo - A GEOGRAPHY or GEOMETRY value. + * srid - The new SRID value of the geography or geometry. + """, + examples = """ + Examples: + > SELECT st_srid(_FUNC_(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)); + 4326 + > SELECT st_srid(_FUNC_(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)); + 3857 + """, + since = "4.1.0", + group = "st_funcs" +) +case class ST_SetSrid(geo: Expression, srid: Expression) + extends RuntimeReplaceable + with ImplicitCastInputTypes + with BinaryLike[Expression] { + + override def inputTypes: Seq[AbstractDataType] = + Seq( + TypeCollection(GeographyType, GeometryType), + IntegerType + ) + + override lazy val replacement: Expression = StaticInvoke( + classOf[STUtils], + STExpressionUtils.geospatialTypeWithSrid(geo.dataType, srid), + "stSetSrid", + Seq(geo, srid), + returnNullable = false + ) + + override def prettyName: String = "st_setsrid" + + override def left: Expression = geo + + override def right: Expression = srid + + override protected def withNewChildrenInternal( + newLeft: Expression, + newRight: Expression): ST_SetSrid = copy(geo = newLeft, srid = newRight) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 55dcea57ff322..a12aa53d1f6f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -665,6 +665,17 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE summary = "") } + def stInvalidSridValueError(srid: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "ST_INVALID_SRID_VALUE", + messageParameters = Map("srid" -> srid) + ) + } + + def stInvalidSridValueError(srid: Int): SparkIllegalArgumentException = { + stInvalidSridValueError(srid.toString) + } + def withSuggestionIntervalArithmeticOverflowError( suggestedFunc: String, context: QueryContext): ArithmeticException = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java index 0e18f8bdf919a..f19a92b61641c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StUtilsSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.util; +import org.apache.spark.SparkIllegalArgumentException; import org.apache.spark.unsafe.types.GeographyVal; import org.apache.spark.unsafe.types.GeometryVal; import org.junit.jupiter.api.Test; @@ -120,4 +121,49 @@ void testStSridGeometry() { assertEquals(testGeometrySrid, STUtils.stSrid(geometryVal)); } + // ST_SetSrid + @Test + void testStSetSridGeography() { + for (int validGeographySrid : new int[]{4326}) { + GeographyVal geographyVal = GeographyVal.fromBytes(testGeographyBytes); + GeographyVal updatedGeographyVal = STUtils.stSetSrid(geographyVal, validGeographySrid); + assertNotNull(updatedGeographyVal); + Geography updatedGeography = Geography.fromBytes(updatedGeographyVal.getBytes()); + assertEquals(validGeographySrid, updatedGeography.srid()); + } + } + + @Test + void testStSetSridGeographyInvalidSrid() { + for (int invalidGeographySrid : new int[]{-9999, -2, -1, 0, 1, 2, 3857, 9999}) { + GeographyVal geographyVal = GeographyVal.fromBytes(testGeographyBytes); + SparkIllegalArgumentException exception = assertThrows(SparkIllegalArgumentException.class, + () -> STUtils.stSetSrid(geographyVal, invalidGeographySrid)); + assertEquals("ST_INVALID_SRID_VALUE", exception.getCondition()); + assertTrue(exception.getMessage().contains("value: " + invalidGeographySrid + ".")); + } + } + + @Test + void testStSetSridGeometry() { + for (int validGeographySrid : new int[]{0, 3857, 4326}) { + GeometryVal geometryVal = GeometryVal.fromBytes(testGeometryBytes); + GeometryVal updatedGeometryVal = STUtils.stSetSrid(geometryVal, validGeographySrid); + assertNotNull(updatedGeometryVal); + Geometry updatedGeometry = Geometry.fromBytes(updatedGeometryVal.getBytes()); + assertEquals(validGeographySrid, updatedGeometry.srid()); + } + } + + @Test + void testStSetSridGeometryInvalidSrid() { + for (int invalidGeometrySrid : new int[]{-9999, -2, -1, 1, 2, 9999}) { + GeometryVal geometryVal = GeometryVal.fromBytes(testGeometryBytes); + SparkIllegalArgumentException exception = assertThrows(SparkIllegalArgumentException.class, + () -> STUtils.stSetSrid(geometryVal, invalidGeometrySrid)); + assertEquals("ST_INVALID_SRID_VALUE", exception.getCondition()); + assertTrue(exception.getMessage().contains("value: " + invalidGeometrySrid + ".")); + } + } + } diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index d9e4a960a2d78..34739ec1c44c9 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -480,6 +480,7 @@ | org.apache.spark.sql.catalyst.expressions.st.ST_AsBinary | st_asbinary | SELECT hex(st_asbinary(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | | org.apache.spark.sql.catalyst.expressions.st.ST_GeogFromWKB | st_geogfromwkb | SELECT hex(st_asbinary(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | | org.apache.spark.sql.catalyst.expressions.st.ST_GeomFromWKB | st_geomfromwkb | SELECT hex(st_asbinary(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040'))) | struct | +| org.apache.spark.sql.catalyst.expressions.st.ST_SetSrid | st_setsrid | SELECT st_srid(st_setsrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)) | struct | | org.apache.spark.sql.catalyst.expressions.st.ST_Srid | st_srid | SELECT st_srid(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040')) | struct | | org.apache.spark.sql.catalyst.expressions.variant.IsVariantNull | is_variant_null | SELECT is_variant_null(parse_json('null')) | struct | | org.apache.spark.sql.catalyst.expressions.variant.ParseJsonExpressionBuilder | parse_json | SELECT parse_json('{"a":1,"b":0.8}') | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out index dfc0b6b89a84f..55bb5f6ab7b00 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out @@ -148,6 +148,90 @@ Aggregate [count(1) AS count(1)#xL] +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)) +-- !query analysis +Project [st_srid(st_setsrid(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040), 4326)) AS st_srid(st_setsrid(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'), 4326))#x] ++- OneRowRelation + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query analysis +Project [st_srid(st_setsrid(st_geomfromwkb(0x0101000000000000000000F03F0000000000000040), 3857)) AS st_srid(st_setsrid(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040'), 3857))#x] ++- OneRowRelation + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 9999)) +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 4326)) <> 4326 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT (st_srid(st_setsrid(st_geogfromwkb(wkb#x), 4326)) = 4326) + +- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 3857)) <> 3857 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT (st_srid(st_setsrid(st_geomfromwkb(wkb#x), 3857)) = 3857) + +- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 3857)) IS NOT NULL +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 9999)) IS NOT NULL +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + -- !query DROP TABLE geodata -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out index dfc0b6b89a84f..55bb5f6ab7b00 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out @@ -148,6 +148,90 @@ Aggregate [count(1) AS count(1)#xL] +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)) +-- !query analysis +Project [st_srid(st_setsrid(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040), 4326)) AS st_srid(st_setsrid(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040'), 4326))#x] ++- OneRowRelation + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query analysis +Project [st_srid(st_setsrid(st_geomfromwkb(0x0101000000000000000000F03F0000000000000040), 3857)) AS st_srid(st_setsrid(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040'), 3857))#x] ++- OneRowRelation + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 9999)) +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 4326)) <> 4326 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT (st_srid(st_setsrid(st_geogfromwkb(wkb#x), 4326)) = 4326) + +- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 3857)) <> 3857 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT (st_srid(st_setsrid(st_geomfromwkb(wkb#x), 3857)) = 3857) + +- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 3857)) IS NOT NULL +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 9999)) IS NOT NULL +-- !query analysis +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + -- !query DROP TABLE geodata -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql index 6785a5b5254b6..b824be2211228 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql @@ -37,5 +37,23 @@ SELECT ST_Srid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040')); SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_GeogFromWKB(wkb)) <> 4326; SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_GeomFromWKB(wkb)) <> 0; +------ ST modifier expressions + +---- ST_SetSrid + +-- 1. Driver-level queries. +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)); +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)); +-- Error handling: invalid SRID. +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)); +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 9999)); + +-- 2. Table-level queries. +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 4326)) <> 4326; +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 3857)) <> 3857; +-- Error handling: invalid SRID. +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 3857)) IS NOT NULL; +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 9999)) IS NOT NULL; + -- Drop the test table. DROP TABLE geodata; diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out index f1e876600a09f..c5b6daea764bb 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out @@ -161,6 +161,98 @@ struct 0 +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)) +-- !query schema +struct +-- !query output +4326 + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query schema +struct +-- !query output +3857 + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 9999)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 4326)) <> 4326 +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 3857)) <> 3857 +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 3857)) IS NOT NULL +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 9999)) IS NOT NULL +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + -- !query DROP TABLE geodata -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out index f1e876600a09f..c5b6daea764bb 100644 --- a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out @@ -161,6 +161,98 @@ struct 0 +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 4326)) +-- !query schema +struct +-- !query output +4326 + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query schema +struct +-- !query output +3857 + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeogFromWKB(X'0101000000000000000000F03F0000000000000040'), 3857)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT ST_Srid(ST_SetSrid(ST_GeomFromWKB(X'0101000000000000000000F03F0000000000000040'), 9999)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 4326)) <> 4326 +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 3857)) <> 3857 +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeogFromWKB(wkb), 3857)) IS NOT NULL +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "3857" + } +} + + +-- !query +SELECT COUNT(*) FROM geodata WHERE ST_Srid(ST_SetSrid(ST_GeomFromWKB(wkb), 9999)) IS NOT NULL +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ST_INVALID_SRID_VALUE", + "sqlState" : "22023", + "messageParameters" : { + "srid" : "9999" + } +} + + -- !query DROP TABLE geodata -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 1b73617d050e8..700f9700ebc90 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.st._ import org.apache.spark.sql.test.SharedSparkSession @@ -33,6 +34,11 @@ class STExpressionsSuite private final val defaultGeometrySrid: Int = ExpressionDefaults.DEFAULT_GEOMETRY_SRID private final val defaultGeometryType: DataType = GeometryType(defaultGeometrySrid) + // Private helper method to assert the data type of a query result. + private def assertType(query: String, expectedDataType: DataType) = { + assert(sql(query).schema.fields.head.dataType.sameType(expectedDataType)) + } + /** ST reader/writer expressions. */ test("ST_AsBinary") { @@ -79,4 +85,126 @@ class STExpressionsSuite checkEvaluation(stSridGeometryNull, null) } + /** ST modifier expressions. */ + + test("ST_SetSrid - expressions") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + val wkb = Hex.unhex(wkbString.getBytes()) + val wkbLiteral = Literal.create(wkb, BinaryType) + val geographyLiteral = ST_GeogFromWKB(wkbLiteral) + val nullGeographyLiteral = Literal.create(null, defaultGeographyType) + val geometryLiteral = ST_GeomFromWKB(wkbLiteral) + val nullGeometryLiteral = Literal.create(null, defaultGeometryType) + val srid = 4326 + val sridLiteral = Literal.create(srid, IntegerType) + val nullSridLiteral = Literal.create(null, IntegerType) + val invalidSrid = 9999 + val invalidSridLiteral = Literal.create(9999, IntegerType) + + // ST_SetSrid on GEOGRAPHY expression. + val geogLit = ST_SetSrid(geographyLiteral, sridLiteral) + assert(geogLit.dataType.sameType(GeographyType(srid))) + checkEvaluation(ST_AsBinary(geogLit), wkb) + val geogLitSrid = ST_Srid(geogLit) + assert(geogLitSrid.dataType.sameType(IntegerType)) + checkEvaluation(geogLitSrid, srid) + // Test NULL handling on GEOGRAPHY. + val nullGeog = ST_SetSrid(nullGeographyLiteral, sridLiteral) + assert(nullGeog.dataType.sameType(GeographyType(srid))) + checkEvaluation(nullGeog, null) + val geogNullSrid = ST_SetSrid(geographyLiteral, nullSridLiteral) + assert(geogNullSrid.dataType.sameType(GeographyType("ANY"))) + checkEvaluation(geogNullSrid, null) + // Test error handling for invalid SRID. + val geogInvalidSrid = ST_SetSrid(geographyLiteral, invalidSridLiteral) + checkError( + exception = intercept[SparkIllegalArgumentException] { + geogInvalidSrid.eval() + }, + condition = "ST_INVALID_SRID_VALUE", + parameters = Map("srid" -> s"$invalidSrid") + ) + + // ST_SetSrid on GEOMETRY expression. + val geomLit = ST_SetSrid(geometryLiteral, sridLiteral) + assert(geomLit.dataType.sameType(GeometryType(srid))) + checkEvaluation(ST_AsBinary(geomLit), wkb) + val geomLitSrid = ST_Srid(geomLit) + assert(geomLitSrid.dataType.sameType(IntegerType)) + checkEvaluation(geomLitSrid, srid) + // Test NULL handling on GEOMETRY. + val nullGeom = ST_SetSrid(nullGeometryLiteral, sridLiteral) + assert(nullGeom.dataType.sameType(GeometryType(srid))) + checkEvaluation(nullGeom, null) + val geomNullSrid = ST_SetSrid(geometryLiteral, nullSridLiteral) + assert(geomNullSrid.dataType.sameType(GeometryType("ANY"))) + checkEvaluation(geomNullSrid, null) + // Test error handling for invalid SRID. + val geomInvalidSrid = ST_SetSrid(geometryLiteral, invalidSridLiteral) + checkError( + exception = intercept[SparkIllegalArgumentException] { + geomInvalidSrid.eval() + }, + condition = "ST_INVALID_SRID_VALUE", + parameters = Map("srid" -> s"$invalidSrid") + ) + } + + test("ST_SetSrid - columns") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + val srid = 4326 + + withTable("tbl") { + // Construct the test table. + sql(s"CREATE TABLE tbl (wkb BINARY, srid INT)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString', $srid)") + + // ST_SetSrid on GEOGRAPHY column value, with SRID column value. + val geogColSridCol = "ST_SetSrid(ST_GeogFromWKB(wkb), srid)" + assertType(s"SELECT $geogColSridCol FROM tbl", GeographyType("ANY")) + assertType(s"SELECT ST_Srid($geogColSridCol) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geogColSridCol) FROM tbl"), Row(srid)) + // ST_SetSrid on GEOMETRY column value, with SRID column value. + val geomColSridCol = "ST_SetSrid(ST_GeomFromWKB(wkb), srid)" + assertType(s"SELECT $geomColSridCol FROM tbl", GeometryType("ANY")) + assertType(s"SELECT ST_Srid($geomColSridCol) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geomColSridCol) FROM tbl"), Row(srid)) + + // ST_SetSrid on GEOGRAPHY literal value, with SRID column value. + val geogLitSridCol = s"ST_SetSrid(ST_GeogFromWKB(X'$wkbString'), srid)" + assertType(s"SELECT $geogLitSridCol FROM tbl", GeographyType("ANY")) + assertType(s"SELECT ST_Srid($geogLitSridCol) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geogLitSridCol) FROM tbl"), Row(srid)) + // ST_SetSrid on GEOMETRY literal value, with SRID column value. + val geomLitSridCol = s"ST_SetSrid(ST_GeomFromWKB(X'$wkbString'), srid)" + assertType(s"SELECT $geomLitSridCol FROM tbl", GeometryType("ANY")) + assertType(s"SELECT ST_Srid($geomLitSridCol) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geomLitSridCol) FROM tbl"), Row(srid)) + + // ST_SetSrid on GEOGRAPHY column value, with SRID literal. + val geogColSridLit = s"ST_SetSrid(ST_GeogFromWKB(wkb), $srid)" + assertType(s"SELECT $geogColSridLit FROM tbl", GeographyType(srid)) + assertType(s"SELECT ST_Srid($geogColSridLit) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geogColSridLit) FROM tbl"), Row(srid)) + // ST_SetSrid on GEOMETRY column value, with SRID literal. + val geomColSridLit = s"ST_SetSrid(ST_GeomFromWKB(wkb), $srid)" + assertType(s"SELECT $geomColSridLit FROM tbl", GeometryType(srid)) + assertType(s"SELECT ST_Srid($geomColSridLit) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geomColSridLit) FROM tbl"), Row(srid)) + + // ST_SetSrid on GEOGRAPHY literal value, with SRID literal. + val geogLitSridLit = s"ST_SetSrid(ST_GeogFromWKB(X'$wkbString'), $srid)" + assertType(s"SELECT $geogLitSridLit FROM tbl", GeographyType(srid)) + assertType(s"SELECT ST_Srid($geogLitSridLit) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geogLitSridLit) FROM tbl"), Row(srid)) + // ST_SetSrid on GEOMETRY literal value, with SRID literal. + val geomLitSridLit = s"ST_SetSrid(ST_GeomFromWKB(X'$wkbString'), $srid)" + assertType(s"SELECT $geomLitSridLit FROM tbl", GeometryType(srid)) + assertType(s"SELECT ST_Srid($geomLitSridLit) FROM tbl", IntegerType) + checkAnswer(sql(s"SELECT ST_Srid($geomLitSridLit) FROM tbl"), Row(srid)) + } + } + } From 2d37bb08a9798d9378d7fc51ce184c0ce167f26c Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Wed, 5 Nov 2025 13:21:23 -0800 Subject: [PATCH 045/400] [SPARK-54178][SQL] Improve error for ResolveSQLOnFile ### What changes were proposed in this pull request? Improve error for ResolveSQLOnFile - generic error does not mean that the data source is not supported! ### Why are the changes needed? Currently `ResolveSQLOnFile` throws `UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY` for a generic failure when discovering files and figuring out the file schemas. This is confusing. We need a separate error. ### Does this PR introduce _any_ user-facing change? Better error message. ### How was this patch tested? Hard to create a test, because the files need to be corrupted. ### Was this patch authored or co-authored using generative AI tooling? Claude. Closes #52875 from vladimirg-db/vladimir-golubev_data/better-error-for-resolve-sql-on-file. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../utils/src/main/resources/error/error-conditions.json | 6 ++++++ .../apache/spark/sql/errors/QueryCompilationErrors.scala | 8 ++++++++ .../apache/spark/sql/execution/datasources/rules.scala | 8 ++------ 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index a34ceb9f11454..55f99208e0a15 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1827,6 +1827,12 @@ ], "sqlState" : "2203G" }, + "FAILED_TO_CREATE_PLAN_FOR_DIRECT_QUERY" : { + "message" : [ + "Failed to create plan for direct query on files: " + ], + "sqlState" : "58030" + }, "FAILED_TO_LOAD_ROUTINE" : { "message" : [ "Failed to load routine ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 7d79c5d5d642d..9847dbd6b197c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1770,6 +1770,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map("provider" -> provider)) } + def failedToCreatePlanForDirectQueryError( + dataSourceType: String, cause: Throwable): Throwable = { + new AnalysisException( + errorClass = "FAILED_TO_CREATE_PLAN_FOR_DIRECT_QUERY", + messageParameters = Map("dataSourceType" -> dataSourceType), + cause = Some(cause)) + } + def findMultipleDataSourceError(provider: String, sourceNames: Seq[String]): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1141", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index c8cb5d7ce7c51..060d7fe72c0a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -65,12 +65,8 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { messageParameters = e.getMessageParameters.asScala.toMap) case _: ClassNotFoundException => None case e: Exception if !e.isInstanceOf[AnalysisException] => - // the provider is valid, but failed to create a logical plan - u.failAnalysis( - errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", - messageParameters = Map("dataSourceType" -> u.multipartIdentifier.head), - cause = e - ) + throw QueryCompilationErrors.failedToCreatePlanForDirectQueryError( + u.multipartIdentifier.head, e) } case _ => None From fb16a1ea6fef6b65a73716362679f4645b8d6e29 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Nov 2025 17:54:27 -0800 Subject: [PATCH 046/400] [SPARK-54197][K8S] Improve `ExecutorsPodsLifecycleManager` not to request to delete if `deletionTimestamp` exists ### What changes were proposed in this pull request? The current code handling deletion of Failed or Succeeded driver Pods is calling the Kubernetes API to delete objects until either the Kubelet as started the termination the Pod (the status of the object is terminating). However, depending on configuration, the ExecutorPodsLifecycleManager loop might run multiple times before the Kubelet starts the deletion of the Pod object, resulting in un-necessary DELETE calls to the Kubernetes API, which are particularly expensive since they are served from Etcd. Following the Kubernetes API specifications in https://kubernetes.io/docs/reference/using-api/api-concepts/ > When a client first sends a delete to request the removal of a resource, the .metadata.deletionTimestamp is set to the current time. Once the .metadata.deletionTimestamp is set, external controllers that act on finalizers may start performing their cleanup work at any time, in any order. we can assume that whenever the deletionTimestamp is set on a Pod, this will be eventually terminated without the need of additional DELETE calls. ### Why are the changes needed? This change is required to remove the need of redundant API calls agains the Kubernetes API that at scale might lead to excessive load against Etcd. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This patch includes unit-tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52898 Closes #52902 from dongjoon-hyun/driver-do-not-call-delete-for-terminating-pods-master. Lead-authored-by: Dongjoon Hyun Co-authored-by: Andrea Tosatto Signed-off-by: Dongjoon Hyun (cherry picked from commit 3b368ca9715c741556bcd38cbe662aeebb79c64d) Signed-off-by: Dongjoon Hyun --- .../k8s/ExecutorPodsLifecycleManager.scala | 16 ++++++++--- .../ExecutorPodsLifecycleManagerSuite.scala | 27 ++++++++++++++++++- 2 files changed, 39 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index 9aa99a6c5984e..2ad01229f8f25 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -201,8 +201,16 @@ private[spark] class ExecutorPodsLifecycleManager( private def removeExecutorFromK8s(execId: Long, updatedPod: Pod): Unit = { Utils.tryLogNonFatalError { if (shouldDeleteExecutors) { - // Get pod before deleting it, we can skip deleting if pod is already deleted so that - // we do not send too many requests to api server. + if (updatedPod.getMetadata.getDeletionTimestamp != null) { + // Do not call the Kubernetes API if the deletion timestamp + // is already set on the updatedPod object. + // This is removing the need for un-necessary API roundtrips + // against the Kubernetes API. + return + } + // Get pod before deleting it, we can skip deleting if pod is already deleted + // or has already the deletion timestamp set so that we do not send + // too many requests to apu server. // If deletion failed on a previous try, we can try again if resync informs us the pod // is still around. // Delete as best attempt - duplicate deletes will throw an exception but the end state @@ -211,7 +219,9 @@ private[spark] class ExecutorPodsLifecycleManager( .pods() .inNamespace(namespace) .withName(updatedPod.getMetadata.getName) - if (podToDelete.get() != null) { + + if (podToDelete.get() != null && + podToDelete.get.getMetadata.getDeletionTimestamp == null) { podToDelete.delete() } } else if (!inactivatedPods.contains(execId) && !isPodInactive(updatedPod)) { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala index 4c7ffe692b105..7d81cc0ae16c6 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala @@ -20,7 +20,7 @@ import java.util.function.UnaryOperator import scala.collection.mutable -import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{Mock, MockitoAnnotations} @@ -219,6 +219,31 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) } + test("Don't delete pod from K8s if deletionTimestamp is already set.") { + // Create a failed pod with deletionTimestamp already in the past + val basePod = failedExecutorWithoutDeletion(1) + val failedPodWithDeletionTimestamp = new PodBuilder(basePod) + .editOrNewMetadata() + .withDeletionTimestamp("1970-01-01T00:00:00Z") + .endMetadata() + .build() + + val mockPodResource = mock(classOf[PodResource]) + namedExecutorPods.put("spark-executor-1", mockPodResource) + when(mockPodResource.get()).thenReturn(failedPodWithDeletionTimestamp) + + snapshotsStore.updatePod(failedPodWithDeletionTimestamp) + snapshotsStore.notifySubscribers() + + // Verify executor is removed from Spark + val msg = "The executor with id 1 was deleted by a user or the framework." + val expectedLossReason = ExecutorExited(1, exitCausedByApp = false, msg) + verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) + + // Verify delete() is NOT called since deletionTimestamp is already set + verify(mockPodResource, never()).delete() + } + private def exitReasonMessage(execId: Int, failedPod: Pod, exitCode: Int): String = { val reason = Option(failedPod.getStatus.getReason) val message = Option(failedPod.getStatus.getMessage) From f737799e15fe668bb0dfc3203a4bca1e7485466e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Nov 2025 18:21:46 -0800 Subject: [PATCH 047/400] [SPARK-54159][INFRA] Fix `pip install` commands to use proper quotation marks ### What changes were proposed in this pull request? This PR aims to a few `pip install` command to use proper quotation marks like the following. ``` -RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.13 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this ``` ``` -RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ +RUN python3.13 -m pip install 'numpy>=2.1' 'pyarrow>=18.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy coverage matplotlib openpyxl 'grpcio==1.67.0' 'grpcio-status==1.67.0' lxml jinja2 && \ ``` ### Why are the changes needed? SHELL handles `>=` before `pip install` command receives it. ### Does this PR introduce _any_ user-facing change? No behavior change because this is only changing infra. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. This patch had conflicts when merged, resolved by Committer: Dongjoon Hyun Closes #52857 from dongjoon-hyun/SPARK-54159. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 37689bfea45f3dd3259d3a60b33469cc9bf248a6) Signed-off-by: Dongjoon Hyun --- dev/create-release/spark-rm/Dockerfile | 2 +- dev/infra/Dockerfile | 12 ++++++------ dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- .../python-311-classic-only/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- dev/spark-test-image/python-313-nogil/Dockerfile | 4 ++-- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- 10 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 679998b893927..af8c34852ca03 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -98,7 +98,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 -RUN python3.10 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.10 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.10 -m pip install --ignore-installed 'six==1.16.0' # Avoid `python3-six` installation RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.10 -m pip install 'torch<2.6.0' torchvision --index-url https://download.pytorch.org/whl/cpu && \ diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 423b6ba820d07..fdfd89bdf06d3 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -101,7 +101,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 -RUN python3.10 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.10 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.10 -m pip install --ignore-installed 'six==1.16.0' # Avoid `python3-six` installation RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.10 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ @@ -114,7 +114,7 @@ RUN apt-get update && apt-get install -y \ python3.9 python3.9-distutils \ && rm -rf /var/lib/apt/lists/* RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 -RUN python3.9 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.9 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.9 -m pip install --force $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.9 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.9 -m pip install torcheval && \ @@ -125,7 +125,7 @@ RUN apt-get update && apt-get install -y \ python3.11 \ && rm -rf /var/lib/apt/lists/* RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 -RUN python3.11 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.11 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.11 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.11 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.11 -m pip install deepspeed torcheval && \ @@ -136,7 +136,7 @@ RUN apt-get update && apt-get install -y \ python3.12 \ && rm -rf /var/lib/apt/lists/* RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 -RUN python3.12 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.12 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.12 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ python3.12 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.12 -m pip install torcheval && \ @@ -148,8 +148,8 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 # TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 -RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl grpcio==1.76.0 grpcio-status==1.76.0 lxml jinja2 && \ +RUN python3.13 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this +RUN python3.13 -m pip install 'numpy>=2.1' 'pyarrow>=18.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy coverage matplotlib openpyxl 'grpcio==1.76.0' 'grpcio-status==1.76.0' lxml jinja2 && \ python3.13 -m pip cache purge # Remove unused installation packages to free up disk space diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index d33fb5f5d30e4..06b21bb555bb9 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -75,6 +75,6 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 -RUN python3.11 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.11 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.11 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.11 -m pip cache purge diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index 46cfce36832bf..99d231f47bd7e 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -70,7 +70,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 -RUN python3.10 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.10 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.10 -m pip install --ignore-installed 'six==1.16.0' # Avoid `python3-six` installation RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.10 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ diff --git a/dev/spark-test-image/python-311-classic-only/Dockerfile b/dev/spark-test-image/python-311-classic-only/Dockerfile index c2d48c5ce877f..484c940988443 100644 --- a/dev/spark-test-image/python-311-classic-only/Dockerfile +++ b/dev/spark-test-image/python-311-classic-only/Dockerfile @@ -73,7 +73,7 @@ ARG TEST_PIP_PKGS="coverage unittest-xml-reporting" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 -RUN python3.11 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.11 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.11 -m pip install $BASIC_PIP_PKGS $TEST_PIP_PKGS && \ python3.11 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.11 -m pip install deepspeed torcheval && \ diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 920f21bd47ee7..a45ab45548480 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -74,7 +74,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 -RUN python3.11 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.11 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.11 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.11 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.11 -m pip install deepspeed torcheval && \ diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index db1039f5cb260..e05fe327a0302 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -74,7 +74,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 -RUN python3.12 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.12 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.12 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ python3.12 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.12 -m pip install torcheval && \ diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index a50bf670b3f35..d4469a9cd4947 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -76,6 +76,6 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13t # TODO: Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS when it supports Python 3.13 free threaded # TODO: Add lxml, grpcio, grpcio-status back when they support Python 3.13 free threaded -RUN python3.13t -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this -RUN python3.13t -m pip install numpy>=2.1 pyarrow>=19.0.0 six==1.16.0 pandas==2.3.3 scipy coverage matplotlib openpyxl jinja2 && \ +RUN python3.13t -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this +RUN python3.13t -m pip install 'numpy>=2.1' 'pyarrow>=19.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy coverage matplotlib openpyxl jinja2 && \ python3.13t -m pip cache purge diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index f74c48bf346f8..1d3d73c9feb9c 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -74,7 +74,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 -RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.13 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.13 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ python3.13 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.13 -m pip install torcheval && \ diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 2a36da897e2df..236e95e3f52cb 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -74,7 +74,7 @@ ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 goog # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 -RUN python3.14 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.14 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this RUN python3.14 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ python3.14 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.14 -m pip install torcheval && \ From f993658460e537948badcc96cdefa0c4e9fac22a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Nov 2025 21:14:26 -0800 Subject: [PATCH 048/400] [SPARK-54213][PYTHON][CONNECT] Remove Python 3.9 from Spark Connect ### What changes were proposed in this pull request? This PR aims to remove `Python 3.9` from `Spark Connect`. ### Why are the changes needed? `Python 3.9` reached the end-of-life on 2025-10-31. - https://devguide.python.org/versions/#unsupported-versions Apache Spark 4.1.0 dropped `Python 3.9` support and we don't have a test coverage. We had better make it clear even that is `Spark Connect` module. - https://github.com/apache/spark/pull/51259 - https://github.com/apache/spark/pull/51416 - https://github.com/apache/spark/pull/51631 - https://github.com/apache/spark/pull/51371 ### Does this PR introduce _any_ user-facing change? No behavior change for Python 3.10+ users. ### How was this patch tested? Pass the CIs and manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52911 from dongjoon-hyun/SPARK-54213. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 4ba1189dabc46a8cb6fb9abb0e0b9b271487984e) Signed-off-by: Dongjoon Hyun --- python/packaging/connect/setup.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 3b88563bcfe7e..bd4d6fc93b6f6 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -124,10 +124,9 @@ "numpy>=%s" % _minimum_numpy_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], - python_requires=">=3.9", + python_requires=">=3.10", classifiers=[ "Development Status :: 5 - Production/Stable", - "Programming Language :: Python :: 3.9", "Programming Language :: Python :: 3.10", "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", From 25a11c4eadef3cf83bd0bade11fbdfae24f57c40 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Nov 2025 21:39:08 -0800 Subject: [PATCH 049/400] Revert "[SPARK-53525][CONNECT][FOLLOWUP] Spark Connect ArrowBatch Result Chunking - Scala Client" This reverts commit bc0f6f7a8a0db3d56b106f23956aa6e6e999d99d. --- .../sql/connect/ClientE2ETestSuite.scala | 159 +----------------- .../sql/connect/test/RemoteSparkSession.scala | 26 +-- .../connect/client/SparkConnectClient.scala | 45 +---- .../sql/connect/client/SparkResult.scala | 138 +++++---------- 4 files changed, 57 insertions(+), 311 deletions(-) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala index 450ff8ca62490..8c336b6fa6d5d 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala @@ -26,14 +26,12 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.{DurationInt, FiniteDuration} import scala.jdk.CollectionConverters._ -import io.grpc.{CallOptions, Channel, ClientCall, ClientInterceptor, ForwardingClientCall, ForwardingClientCallListener, MethodDescriptor} import org.apache.commons.io.output.TeeOutputStream import org.scalactic.TolerantNumerics import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkArithmeticException, SparkException, SparkUpgradeException} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} -import org.apache.spark.connect.proto import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.sql.{functions, AnalysisException, Observation, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, TableAlreadyExistsException, TempTableAlreadyExistsException} @@ -43,7 +41,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connect.ConnectConversions._ import org.apache.spark.sql.connect.client.{RetryPolicy, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.test.{ConnectFunSuite, IntegrationTestUtils, QueryTest, RemoteSparkSession, SQLHelper} -import org.apache.spark.sql.connect.test.SparkConnectServerUtils.{createSparkSession, port} +import org.apache.spark.sql.connect.test.SparkConnectServerUtils.port import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ @@ -1850,161 +1848,6 @@ class ClientE2ETestSuite checkAnswer(df, Seq.empty) } } - - // Helper class to capture Arrow batch chunk information from gRPC responses - private class ArrowBatchInterceptor extends ClientInterceptor { - case class BatchInfo( - batchIndex: Int, - rowCount: Long, - startOffset: Long, - chunks: Seq[ChunkInfo]) { - def totalChunks: Int = chunks.length - } - - case class ChunkInfo( - batchIndex: Int, - chunkIndex: Int, - numChunksInBatch: Int, - rowCount: Long, - startOffset: Long, - dataSize: Int) - - private val batches: mutable.Buffer[BatchInfo] = mutable.Buffer.empty - private var currentBatchIndex: Int = 0 - private val currentBatchChunks: mutable.Buffer[ChunkInfo] = mutable.Buffer.empty - - override def interceptCall[ReqT, RespT]( - method: MethodDescriptor[ReqT, RespT], - callOptions: CallOptions, - next: Channel): ClientCall[ReqT, RespT] = { - new ForwardingClientCall.SimpleForwardingClientCall[ReqT, RespT]( - next.newCall(method, callOptions)) { - override def start( - responseListener: ClientCall.Listener[RespT], - headers: io.grpc.Metadata): Unit = { - super.start( - new ForwardingClientCallListener.SimpleForwardingClientCallListener[RespT]( - responseListener) { - override def onMessage(message: RespT): Unit = { - message match { - case response: proto.ExecutePlanResponse if response.hasArrowBatch => - val arrowBatch = response.getArrowBatch - // Track chunk information for every chunk - currentBatchChunks += ChunkInfo( - batchIndex = currentBatchIndex, - chunkIndex = arrowBatch.getChunkIndex.toInt, - numChunksInBatch = arrowBatch.getNumChunksInBatch.toInt, - rowCount = arrowBatch.getRowCount, - startOffset = arrowBatch.getStartOffset, - dataSize = arrowBatch.getData.size()) - // When we receive the last chunk, create the BatchInfo - if (currentBatchChunks.length == arrowBatch.getNumChunksInBatch) { - batches += BatchInfo( - batchIndex = currentBatchIndex, - rowCount = arrowBatch.getRowCount, - startOffset = arrowBatch.getStartOffset, - chunks = currentBatchChunks.toList) - currentBatchChunks.clear() - currentBatchIndex += 1 - } - case _ => // Not an ExecutePlanResponse with ArrowBatch, ignore - } - super.onMessage(message) - } - }, - headers) - } - } - } - - // Get all batch information - def getBatchInfos: Seq[BatchInfo] = batches.toSeq - - def clear(): Unit = { - currentBatchIndex = 0 - currentBatchChunks.clear() - batches.clear() - } - } - - test("Arrow batch result chunking") { - // This test validates that the client can correctly reassemble chunked Arrow batches - // using SequenceInputStream as implemented in SparkResult.processResponses - - // Two cases are tested here: - // (a) client preferred chunk size is set: the server should respect it - // (b) client preferred chunk size is not set: the server should use its own max chunk size - Seq((Some(1024), None), (None, Some(1024))).foreach { - case (preferredChunkSizeOpt, maxChunkSizeOpt) => - // Create interceptor to capture chunk information - val arrowBatchInterceptor = new ArrowBatchInterceptor() - - try { - // Set preferred chunk size if specified and add interceptor - preferredChunkSizeOpt match { - case Some(size) => - spark = createSparkSession( - _.preferredArrowChunkSize(Some(size)).interceptor(arrowBatchInterceptor)) - case None => - spark = createSparkSession(_.interceptor(arrowBatchInterceptor)) - } - // Set server max chunk size if specified - maxChunkSizeOpt.foreach { size => - spark.conf.set("spark.connect.session.resultChunking.maxChunkSize", size.toString) - } - - val sqlQuery = - "select id, CAST(id + 0.5 AS DOUBLE) as double_val from range(0, 2000, 1, 4)" - - // Execute the query using withResult to access SparkResult object - spark.sql(sqlQuery).withResult { result => - // Verify the results are correct and complete - assert(result.length == 2000) - - // Get batch information from interceptor - val batchInfos = arrowBatchInterceptor.getBatchInfos - - // Assert there are 4 batches (partitions) in total - assert(batchInfos.length == 4) - - // Validate chunk information for each batch - val maxChunkSize = preferredChunkSizeOpt.orElse(maxChunkSizeOpt).get - batchInfos.foreach { batch => - // In this example, the max chunk size is set to a small value, - // so each Arrow batch should be split into multiple chunks - assert(batch.totalChunks > 5) - assert(batch.chunks.nonEmpty) - assert(batch.chunks.length == batch.totalChunks) - batch.chunks.zipWithIndex.foreach { case (chunk, expectedIndex) => - assert(chunk.chunkIndex == expectedIndex) - assert(chunk.numChunksInBatch == batch.totalChunks) - assert(chunk.rowCount == batch.rowCount) - assert(chunk.startOffset == batch.startOffset) - assert(chunk.dataSize > 0) - assert(chunk.dataSize <= maxChunkSize) - } - } - - // Validate data integrity across the range to ensure chunking didn't corrupt anything - val rows = result.toArray - var expectedId = 0L - rows.foreach { row => - assert(row.getLong(0) == expectedId) - val expectedDouble = expectedId + 0.5 - val actualDouble = row.getDouble(1) - assert(math.abs(actualDouble - expectedDouble) < 0.001) - expectedId += 1 - } - } - } finally { - // Clean up configurations - maxChunkSizeOpt.foreach { _ => - spark.conf.unset("spark.connect.session.resultChunking.maxChunkSize") - } - arrowBatchInterceptor.clear() - } - } - } } private[sql] case class ClassData(a: String, b: Int) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala index a239775a3a86b..efb6c721876c3 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala @@ -187,27 +187,19 @@ object SparkConnectServerUtils { } def createSparkSession(): SparkSession = { - createSparkSession(identity) - } - - def createSparkSession( - customBuilderFunc: SparkConnectClient.Builder => SparkConnectClient.Builder) - : SparkSession = { SparkConnectServerUtils.start() - var builder = SparkConnectClient - .builder() - .userId("test") - .port(port) - .retryPolicy( - RetryPolicy - .defaultPolicy() - .copy(maxRetries = Some(10), maxBackoff = Some(FiniteDuration(30, "s")))) - - builder = customBuilderFunc(builder) val spark = SparkSession .builder() - .client(builder.build()) + .client( + SparkConnectClient + .builder() + .userId("test") + .port(port) + .retryPolicy(RetryPolicy + .defaultPolicy() + .copy(maxRetries = Some(10), maxBackoff = Some(FiniteDuration(30, "s")))) + .build()) .create() // Execute an RPC which will get retried until the server is up. diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index e5fd16a7c2612..fa32eba91eb2c 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -138,22 +138,6 @@ private[sql] class SparkConnectClient( .setSessionId(sessionId) .setClientType(userAgent) .addAllTags(tags.get.toSeq.asJava) - - // Add request option to allow result chunking. - if (configuration.allowArrowBatchChunking) { - val chunkingOptionsBuilder = proto.ResultChunkingOptions - .newBuilder() - .setAllowArrowBatchChunking(true) - configuration.preferredArrowChunkSize.foreach { size => - chunkingOptionsBuilder.setPreferredArrowChunkSize(size) - } - request.addRequestOptions( - proto.ExecutePlanRequest.RequestOption - .newBuilder() - .setResultChunkingOptions(chunkingOptionsBuilder.build()) - .build()) - } - serverSideSessionId.foreach(session => request.setClientObservedServerSideSessionId(session)) operationId.foreach { opId => require( @@ -348,16 +332,6 @@ private[sql] class SparkConnectClient( def copy(): SparkConnectClient = configuration.toSparkConnectClient - /** - * Returns whether arrow batch chunking is allowed. - */ - def allowArrowBatchChunking: Boolean = configuration.allowArrowBatchChunking - - /** - * Returns the preferred arrow chunk size in bytes. - */ - def preferredArrowChunkSize: Option[Int] = configuration.preferredArrowChunkSize - /** * Add a single artifact to the client session. * @@ -783,21 +757,6 @@ object SparkConnectClient { this } - def allowArrowBatchChunking(allow: Boolean): Builder = { - _configuration = _configuration.copy(allowArrowBatchChunking = allow) - this - } - - def allowArrowBatchChunking: Boolean = _configuration.allowArrowBatchChunking - - def preferredArrowChunkSize(size: Option[Int]): Builder = { - size.foreach(s => require(s > 0, "preferredArrowChunkSize must be positive")) - _configuration = _configuration.copy(preferredArrowChunkSize = size) - this - } - - def preferredArrowChunkSize: Option[Int] = _configuration.preferredArrowChunkSize - def build(): SparkConnectClient = _configuration.toSparkConnectClient } @@ -842,9 +801,7 @@ object SparkConnectClient { interceptors: List[ClientInterceptor] = List.empty, sessionId: Option[String] = None, grpcMaxMessageSize: Int = ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE, - grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT, - allowArrowBatchChunking: Boolean = true, - preferredArrowChunkSize: Option[Int] = None) { + grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) { private def isLocal = host.equals("localhost") diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 43265e55a0ca9..ef55edd10c8a3 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -16,21 +16,18 @@ */ package org.apache.spark.sql.connect.client -import java.io.SequenceInputStream import java.lang.ref.Cleaner import java.util.Objects import scala.collection.mutable import scala.jdk.CollectionConverters._ -import com.google.protobuf.ByteString import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch} import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics -import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} @@ -45,8 +42,7 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String) - extends AutoCloseable - with Logging { self => + extends AutoCloseable { self => case class StageInfo( stageId: Long, @@ -122,7 +118,6 @@ private[sql] class SparkResult[T]( stopOnFirstNonEmptyResponse: Boolean = false): Boolean = { var nonEmpty = false var stop = false - val arrowBatchChunksToAssemble = mutable.Buffer.empty[ByteString] while (!stop && responses.hasNext) { val response = responses.next() @@ -156,96 +151,55 @@ private[sql] class SparkResult[T]( stop |= stopOnSchema } if (response.hasArrowBatch) { - val arrowBatch = response.getArrowBatch - logDebug( - s"Received arrow batch rows=${arrowBatch.getRowCount} " + - s"Number of chunks in batch=${arrowBatch.getNumChunksInBatch} " + - s"Chunk index=${arrowBatch.getChunkIndex} " + - s"size=${arrowBatch.getData.size()}") - - if (arrowBatchChunksToAssemble.nonEmpty) { - // Expect next chunk of the same batch - if (arrowBatch.getChunkIndex != arrowBatchChunksToAssemble.size) { - throw new IllegalStateException( - s"Expected chunk index ${arrowBatchChunksToAssemble.size} of the " + - s"arrow batch but got ${arrowBatch.getChunkIndex}.") - } - } else { - // Expect next batch - if (arrowBatch.hasStartOffset) { - val expectedStartOffset = arrowBatch.getStartOffset - if (numRecords != expectedStartOffset) { - throw new IllegalStateException( - s"Expected arrow batch to start at row offset $numRecords in results, " + - s"but received arrow batch starting at offset $expectedStartOffset.") - } - } - if (arrowBatch.getChunkIndex != 0) { - throw new IllegalStateException( - s"Expected chunk index 0 of the next arrow batch " + - s"but got ${arrowBatch.getChunkIndex}.") - } + val ipcStreamBytes = response.getArrowBatch.getData + val expectedNumRows = response.getArrowBatch.getRowCount + val reader = new MessageIterator(ipcStreamBytes.newInput(), allocator) + if (arrowSchema == null) { + arrowSchema = reader.schema + stop |= stopOnArrowSchema + } else if (arrowSchema != reader.schema) { + throw new IllegalStateException( + s"""Schema Mismatch between expected and received schema: + |=== Expected Schema === + |$arrowSchema + |=== Received Schema === + |${reader.schema} + |""".stripMargin) } - - arrowBatchChunksToAssemble += arrowBatch.getData - - // Assemble the chunks to an arrow batch to process if - // (a) chunking is not enabled (numChunksInBatch is not set or is 0, - // in this case, it is the single chunk in the batch) - // (b) or the client has received all chunks of the batch. - if (!arrowBatch.hasNumChunksInBatch || - arrowBatch.getNumChunksInBatch == 0 || - arrowBatchChunksToAssemble.size == arrowBatch.getNumChunksInBatch) { - - val numChunks = arrowBatchChunksToAssemble.size - val inputStreams = - arrowBatchChunksToAssemble.map(_.newInput()).iterator.asJavaEnumeration - val input = new SequenceInputStream(inputStreams) - arrowBatchChunksToAssemble.clear() - logDebug(s"Assembling arrow batch from $numChunks chunks.") - - val expectedNumRows = arrowBatch.getRowCount - val reader = new MessageIterator(input, allocator) - if (arrowSchema == null) { - arrowSchema = reader.schema - stop |= stopOnArrowSchema - } else if (arrowSchema != reader.schema) { - throw new IllegalStateException( - s"""Schema Mismatch between expected and received schema: - |=== Expected Schema === - |$arrowSchema - |=== Received Schema === - |${reader.schema} - |""".stripMargin) - } - if (structType == null) { - // If the schema is not available yet, fallback to the arrow schema. - structType = ArrowUtils.fromArrowSchema(reader.schema) - } - - var numRecordsInBatch = 0 - val messages = Seq.newBuilder[ArrowMessage] - while (reader.hasNext) { - val message = reader.next() - message match { - case batch: ArrowRecordBatch => - numRecordsInBatch += batch.getLength - case _ => - } - messages += message - } - if (numRecordsInBatch != expectedNumRows) { + if (structType == null) { + // If the schema is not available yet, fallback to the arrow schema. + structType = ArrowUtils.fromArrowSchema(reader.schema) + } + if (response.getArrowBatch.hasStartOffset) { + val expectedStartOffset = response.getArrowBatch.getStartOffset + if (numRecords != expectedStartOffset) { throw new IllegalStateException( - s"Expected $expectedNumRows rows in arrow batch but got $numRecordsInBatch.") + s"Expected arrow batch to start at row offset $numRecords in results, " + + s"but received arrow batch starting at offset $expectedStartOffset.") } - // Skip the entire result if it is empty. - if (numRecordsInBatch > 0) { - numRecords += numRecordsInBatch - resultMap.put(nextResultIndex, (reader.bytesRead, messages.result())) - nextResultIndex += 1 - nonEmpty |= true - stop |= stopOnFirstNonEmptyResponse + } + var numRecordsInBatch = 0 + val messages = Seq.newBuilder[ArrowMessage] + while (reader.hasNext) { + val message = reader.next() + message match { + case batch: ArrowRecordBatch => + numRecordsInBatch += batch.getLength + case _ => } + messages += message + } + if (numRecordsInBatch != expectedNumRows) { + throw new IllegalStateException( + s"Expected $expectedNumRows rows in arrow batch but got $numRecordsInBatch.") + } + // Skip the entire result if it is empty. + if (numRecordsInBatch > 0) { + numRecords += numRecordsInBatch + resultMap.put(nextResultIndex, (reader.bytesRead, messages.result())) + nextResultIndex += 1 + nonEmpty |= true + stop |= stopOnFirstNonEmptyResponse } } } From 972dec74e183502754a327e06742bd1c0a7b078a Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Wed, 5 Nov 2025 21:59:58 -0800 Subject: [PATCH 050/400] [SPARK-54101][GEO][FOLLOWUP] Re-enable Scala/Python parity check for ST functions ### What changes were proposed in this pull request? Re-enable Scala/Python parity check for ST geospatial functions in `test_function_parity`. ### Why are the changes needed? The test was temporarily disabled in https://github.com/apache/spark/pull/52803, but the corresponding functions have been subsequently added on PySpark side as part of https://github.com/apache/spark/pull/52849. ### Does this PR introduce _any_ user-facing change? Yes, casting `GEOGRAPHY()` to `GEOGRAPHY(ANY)` is now allowed. ### How was this patch tested? Existing tests suffice: - `test_functions` Closes #52907 from uros-db/geo-function_parity. Authored-by: Uros Bojanic Signed-off-by: Dongjoon Hyun (cherry picked from commit b284a2c50e9614979322d6a70dd8c346fdf507d8) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_functions.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 07c0c9aaf5242..0f646b3da549a 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -80,10 +80,6 @@ def test_function_parity(self): missing_in_py = jvm_fn_set.difference(py_fn_set) - # Temporarily disable Scala/Python parity check for ST geospatial functions, while the - # feature is under development. Once the geospatial module is stable, remove this. - missing_in_py = {fn for fn in missing_in_py if not fn.startswith("st_")} - # Functions that we expect to be missing in python until they are added to pyspark expected_missing_in_py = set() From f71cd30caadefedec9e4a771ed3e6ec8ac5f84b4 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 6 Nov 2025 17:54:05 +0900 Subject: [PATCH 051/400] [SPARK-54200][SS] Call close() against underlying InputPartition when LowLatencyReaderWrap.close() is called ### What changes were proposed in this pull request? This PR proposes to fix the bug of missing close() on underlying InputPartition when LowLatencyReaderWrap.close() is called. ### Why are the changes needed? Not closing the underlying InputPartition could leak resource; e.g. Kafka consumer is not returned to the pool, which ends up with destroying the purpose of connection pool and creating Kafka consumer instances every batch. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? A new UT for Kafka rather than general one, since Kafka data source has an internal metric to provide the necessary information for validation. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52903 from HeartSaVioR/SPARK-54200. Authored-by: Jungtaek Lim Signed-off-by: Jungtaek Lim (cherry picked from commit 2f9581781ff67be2a6c10be67e4f3936192c916d) Signed-off-by: Jungtaek Lim --- .../consumer/InternalKafkaConsumerPool.scala | 8 ++ .../kafka010/consumer/KafkaDataConsumer.scala | 4 + .../sql/kafka010/KafkaRealTimeModeSuite.scala | 99 ++++++++++++++++++- .../v2/RealTimeStreamScanExec.scala | 4 +- 4 files changed, 112 insertions(+), 3 deletions(-) diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala index edd5121cfbeee..06ccd7548a040 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala @@ -129,6 +129,14 @@ private[consumer] class InternalKafkaConsumerPool( def size(key: CacheKey): Int = numIdle(key) + numActive(key) + private[kafka010] def numActiveInGroupIdPrefix(groupIdPrefix: String): Int = { + import scala.jdk.CollectionConverters._ + + pool.getNumActivePerKey().asScala.filter { case (key, _) => + key.startsWith(groupIdPrefix + "-") + }.values.map(_.toInt).sum + } + // TODO: revisit the relation between CacheKey and kafkaParams - for now it looks a bit weird // as we force all consumers having same (groupId, topicPartition) to have same kafkaParams // which might be viable in performance perspective (kafkaParams might be too huge to use diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala index af4e5bab2947d..126434625a8d5 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala @@ -848,4 +848,8 @@ private[kafka010] object KafkaDataConsumer extends Logging { new KafkaDataConsumer(topicPartition, kafkaParams, consumerPool, fetchedDataPool) } + + private[kafka010] def getActiveSizeInConsumerPool(groupIdPrefix: String): Int = { + consumerPool.numActiveInGroupIdPrefix(groupIdPrefix) + } } diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRealTimeModeSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRealTimeModeSuite.scala index 83aae64d84f7e..468d1da7f467f 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRealTimeModeSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRealTimeModeSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.kafka010 +import java.util.UUID + import org.scalatest.matchers.should.Matchers import org.scalatest.time.SpanSugar._ @@ -26,6 +28,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemorySink import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.kafka010.consumer.KafkaDataConsumer import org.apache.spark.sql.streaming.{StreamingQuery, Trigger} import org.apache.spark.sql.streaming.OutputMode.Update import org.apache.spark.sql.streaming.util.GlobalSingletonManualClock @@ -39,9 +42,7 @@ class KafkaRealTimeModeSuite override protected val defaultTrigger = RealTimeTrigger.apply("3 seconds") override protected def sparkConf: SparkConf = { - // Should turn to use StreamingShuffleManager when it is ready. super.sparkConf - .set("spark.databricks.streaming.realTimeMode.enabled", "true") .set( SQLConf.STATE_STORE_PROVIDER_CLASS, classOf[RocksDBStateStoreProvider].getName) @@ -679,3 +680,97 @@ class KafkaRealTimeModeSuite ) } } + +class KafkaConsumerPoolRealTimeModeSuite + extends KafkaSourceTest + with Matchers { + override protected val defaultTrigger = RealTimeTrigger.apply("3 seconds") + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set( + SQLConf.STATE_STORE_PROVIDER_CLASS, + classOf[RocksDBStateStoreProvider].getName) + } + + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set( + SQLConf.STREAMING_REAL_TIME_MODE_MIN_BATCH_DURATION, + defaultTrigger.batchDurationMs + ) + } + + test("SPARK-54200: Kafka consumers in consumer pool should be properly reused") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 2) + + testUtils.sendMessages(topic, Array("1", "2"), Some(0)) + testUtils.sendMessages(topic, Array("3"), Some(1)) + + val groupIdPrefix = UUID.randomUUID().toString + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("groupIdPrefix", groupIdPrefix) + .load() + .selectExpr("CAST(value AS STRING)") + .as[String] + .map(_.toInt) + .map(_ + 1) + + // At any point of time, Kafka consumer pool should only contain at most 2 active instances. + testStream(reader, Update, sink = new ContinuousMemorySink())( + StartStream(), + CheckAnswerWithTimeout(60000, 2, 3, 4), + WaitUntilCurrentBatchProcessed, + // After completion of batch 0 + new ExternalAction() { + override def runAction(): Unit = { + assertActiveSizeOnConsumerPool(groupIdPrefix, 2) + + testUtils.sendMessages(topic, Array("4", "5"), Some(0)) + testUtils.sendMessages(topic, Array("6"), Some(1)) + } + }, + CheckAnswerWithTimeout(5000, 2, 3, 4, 5, 6, 7), + WaitUntilCurrentBatchProcessed, + // After completion of batch 1 + new ExternalAction() { + override def runAction(): Unit = { + assertActiveSizeOnConsumerPool(groupIdPrefix, 2) + + testUtils.sendMessages(topic, Array("7"), Some(1)) + } + }, + CheckAnswerWithTimeout(5000, 2, 3, 4, 5, 6, 7, 8), + WaitUntilCurrentBatchProcessed, + // After completion of batch 2 + new ExternalAction() { + override def runAction(): Unit = { + assertActiveSizeOnConsumerPool(groupIdPrefix, 2) + } + }, + StopStream + ) + } + + /** + * NOTE: This method leverages that we run test code, driver and executor in a same process in + * a normal unit test setup (say, local[] in spark master). With that setup, we + * can access singleton object directly. + */ + private def assertActiveSizeOnConsumerPool( + groupIdPrefix: String, + maxAllowedActiveSize: Int): Unit = { + val activeSize = KafkaDataConsumer.getActiveSizeInConsumerPool(groupIdPrefix) + assert(activeSize <= maxAllowedActiveSize, s"Consumer pool size is expected to be less " + + s"than $maxAllowedActiveSize, but $activeSize.") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RealTimeStreamScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RealTimeStreamScanExec.scala index c4e072f184e6a..3432f28e12cc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RealTimeStreamScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RealTimeStreamScanExec.scala @@ -83,7 +83,9 @@ case class LowLatencyReaderWrap( reader.get() } - override def close(): Unit = {} + override def close(): Unit = { + reader.close() + } } /** From 425fbafed1e1d27948339f0c9a3b2a802fc9e4d1 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Thu, 6 Nov 2025 20:51:34 +0800 Subject: [PATCH 052/400] [SPARK-54177][BUILD][FOLLOWUP] Restore Guava exclusion from connect-common shading artifacts MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR restores the Guava exclusion from the `connect-common` module shading artifacts ### Why are the changes needed? To workaround a Maven-specific tricky classpath issue, see the manual verification steps below for details. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GHA Maven workflow runs multiple connect modules together `-pl sql/connect/client/jdbc,sql/connect/client/jvm,sql/connect/common,sql/connect/server` so it is not affected by this. This must be verified manually: ``` build/mvn -Phive clean install -DskipTests ``` run `connect-common` and `connect-client-jvm` tests together, succeeded on both the master and this PR ``` build/mvn -Phive -pl sql/connect/client/jvm,sql/connect/common test -fae -DwildcardSuites=none -Dtest=org.apache.spark.sql.JavaEncoderSuite ``` run single `connect-client-jvm` tests, failed on the master, succeeded on this PR ``` build/mvn -Phive -pl sql/connect/client/jvm test -fae -DwildcardSuites=none -Dtest=org.apache.spark.sql.JavaEncoderSuite ``` ``` [ERROR] Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 0.369 s <<< FAILURE! -- in org.apache.spark.sql.JavaEncoderSuite [ERROR] org.apache.spark.sql.JavaEncoderSuite -- Time elapsed: 0.369 s <<< ERROR! java.lang.NoClassDefFoundError: org/sparkproject/guava/cache/CacheLoader at org.apache.spark.sql.connect.test.SparkConnectServerUtils$.createSparkSession(RemoteSparkSession.scala:208) at org.apache.spark.sql.connect.test.SparkConnectServerUtils$.createSparkSession(RemoteSparkSession.scala:190) at org.apache.spark.sql.connect.test.SparkConnectServerUtils.createSparkSession(RemoteSparkSession.scala) at org.apache.spark.sql.JavaEncoderSuite.setup(JavaEncoderSuite.java:45) Caused by: java.lang.ClassNotFoundException: org.sparkproject.guava.cache.CacheLoader at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 4 more [INFO] [INFO] Results: [INFO] [ERROR] Errors: [ERROR] JavaEncoderSuite.setup:45 » NoClassDefFound org/sparkproject/guava/cache/CacheLoader [INFO] [ERROR] Tests run: 1, Failures: 0, Errors: 1, Skipped: 0 ``` ``` [INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 23.00 s -- in org.apache.spark.sql.JavaEncoderSuite [INFO] [INFO] Results: [INFO] [INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0 ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52918 from pan3793/SPARK-54177-followup. Authored-by: Cheng Pan Signed-off-by: yangjie01 (cherry picked from commit b3abaecf30ec3030a0ff4081e711b1a285b06352) Signed-off-by: yangjie01 --- sql/connect/common/pom.xml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 2713f23c51e6a..7e91b056de551 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -121,6 +121,31 @@ + + org.apache.maven.plugins + maven-shade-plugin + + + false + + + org.spark-project.spark:unused + + + + + + package + + shade + + + + From e33cbde5908716b2099239f16e8d666fbcf36c12 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Thu, 6 Nov 2025 07:46:34 -0800 Subject: [PATCH 053/400] [SPARK-54198][K8S] Delete Kubernetes executor pods only once per event processing interval ### What changes were proposed in this pull request? When `ExecutorPodsLifecycleManager` processes the sequence of snapshots in `onNewSnapshots()` it maintains the `execIdsRemovedInThisRound` set of executor ids to not try deleting an executor pod multiple times. But this logic seems to have a flaw because it depends on `onFinalNonDeletedState()`, which depends on `removeExecutorFromSpark()`, which depends on if the executor has been added to `removedExecutorsCache`. Consider the following scenario: 1. `onNewSnapshots()` runs and an executor is selected for deletion due to a `PodFailed` or `PodSucceeded` event. Because `removedExecutorsCache` (3 minutes cache) doesn't contain the executor `onFinalNonDeletedState()` returns with true and the executor is added to `execIdsRemovedInThisRound`. Before adding the executor to `execIdsRemovedInThisRound` `onFinalNonDeletedState()` calls `removeExecutorFromK8s()` to delete the Kubernetes pod. Due to the executor is in `execIdsRemovedInThisRound` the pod deletion is tried only once regardless how many snapshots we process in `onNewSnapshots()`. 2. Let's suppose the pod deletion failed and `spark.kubernetes.executor.eventProcessingInterval` later (1s by default) `onNewSnapshots()` runs again. Because the executor is already in `removedExecutorsCache`, it is never added to `execIdsRemovedInThisRound`, which results in trying to delete the pod as many times as the number of snapshots we process in `onNewSnapshots()`. In our case the pod initial deletion failed due to flooding the kubernetes API so we issued more and more deletes... ### Why are the changes needed? Fix the above scenario. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52899 from peter-toth/SPARK-54198-fix-multiple-pod-deletes. Authored-by: Peter Toth Signed-off-by: Dongjoon Hyun (cherry picked from commit 2ec7439f51f310cff2339c43648100d07318e6ee) Signed-off-by: Dongjoon Hyun --- .../k8s/ExecutorPodsLifecycleManager.scala | 8 +++----- .../k8s/ExecutorPodsLifecycleManagerSuite.scala | 16 ++++++++++++++++ 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index 2ad01229f8f25..3a508add6ccf0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -113,25 +113,23 @@ private[spark] class ExecutorPodsLifecycleManager( inactivatedPods -= execId case deleted@PodDeleted(_) => + execIdsRemovedInThisRound += execId if (removeExecutorFromSpark(schedulerBackend, deleted, execId)) { - execIdsRemovedInThisRound += execId logDebug(s"Snapshot reported deleted executor with id $execId," + s" pod name ${state.pod.getMetadata.getName}") } inactivatedPods -= execId case failed@PodFailed(_) => - val deleteFromK8s = !execIdsRemovedInThisRound.contains(execId) + val deleteFromK8s = execIdsRemovedInThisRound.add(execId) if (onFinalNonDeletedState(failed, execId, schedulerBackend, deleteFromK8s)) { - execIdsRemovedInThisRound += execId logDebug(s"Snapshot reported failed executor with id $execId," + s" pod name ${state.pod.getMetadata.getName}") } case succeeded@PodSucceeded(_) => - val deleteFromK8s = !execIdsRemovedInThisRound.contains(execId) + val deleteFromK8s = execIdsRemovedInThisRound.add(execId) if (onFinalNonDeletedState(succeeded, execId, schedulerBackend, deleteFromK8s)) { - execIdsRemovedInThisRound += execId if (schedulerBackend.isExecutorActive(execId.toString)) { logInfo(log"Snapshot reported succeeded executor with id " + log"${MDC(LogKeys.EXECUTOR_ID, execId)}, even though the application has not " + diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala index 7d81cc0ae16c6..cdbcae050ceb9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala @@ -244,6 +244,22 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte verify(mockPodResource, never()).delete() } + test("SPARK-54198: Delete Kubernetes executor pods only once per event processing interval") { + val failedPod = failedExecutorWithoutDeletion(1) + val mockPodResource = mock(classOf[PodResource]) + namedExecutorPods.put("spark-executor-1", mockPodResource) + when(mockPodResource.get()).thenReturn(failedPod) + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() + snapshotsStore.updatePod(failedPod) + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() + val msg = exitReasonMessage(1, failedPod, 1) + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend, times(1)).doRemoveExecutor("1", expectedLossReason) + verify(namedExecutorPods(failedPod.getMetadata.getName), times(2)).delete() + } + private def exitReasonMessage(execId: Int, failedPod: Pod, exitCode: Int): String = { val reason = Option(failedPod.getStatus.getReason) val message = Option(failedPod.getStatus.getMessage) From eac2a9f420fb72c3d676089fbf2e5a80ca084968 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Thu, 6 Nov 2025 10:33:18 -0800 Subject: [PATCH 054/400] [SPARK-54169][GEO][SQL] Introduce Geography and Geometry types in Arrow writer ### What changes were proposed in this pull request? Add Arrow serialization/deserialization support for `Geography` and `Geometry` types. ### Why are the changes needed? Supporting geospatial types for clients (Spark Connect / Thrift Server / etc.) which consume result sets in Arrow format. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit tests: - `ArrowUtilsSuite` - `ArrowWriterSuite` - `ArrowEncoderSuite` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52863 from uros-db/geo-arrow-serde. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit d6f1e224df8fc620ab44817e4c1034f990f5d8ff) Signed-off-by: Wenchen Fan --- .../org/apache/spark/util/MaybeNull.scala | 36 +++ .../apache/spark/sql/util/ArrowUtils.scala | 77 +++++ .../spark/sql/catalyst/util/STUtils.java | 4 + .../sql/vectorized/ArrowColumnVector.java | 27 ++ .../sql/execution/arrow/ArrowWriter.scala | 47 +++ .../spark/sql/util/ArrowUtilsSuite.scala | 4 + .../client/arrow/ArrowEncoderSuite.scala | 16 +- sql/core/pom.xml | 7 + .../execution/arrow/ArrowWriterSuite.scala | 277 +++++++++++++++++- 9 files changed, 478 insertions(+), 17 deletions(-) create mode 100644 common/utils/src/test/scala/org/apache/spark/util/MaybeNull.scala diff --git a/common/utils/src/test/scala/org/apache/spark/util/MaybeNull.scala b/common/utils/src/test/scala/org/apache/spark/util/MaybeNull.scala new file mode 100644 index 0000000000000..44bdffeacfde6 --- /dev/null +++ b/common/utils/src/test/scala/org/apache/spark/util/MaybeNull.scala @@ -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.spark.util + +/* The MaybeNull class is a utility that introduces controlled nullability into a sequence + * of invocations. It is designed to return a ~null~ value at a specified interval while returning + * the provided value for all other invocations. + */ +case class MaybeNull(interval: Int) { + assert(interval > 1) + private var invocations = 0 + def apply[T](value: T): T = { + val result = if (invocations % interval == 0) { + null.asInstanceOf[T] + } else { + value + } + invocations += 1 + result + } +} diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index 6caabf20f8f6b..23d8a0bbb65b5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -143,6 +143,43 @@ private[sql] object ArrowUtils { largeVarTypes)).asJava) case udt: UserDefinedType[_] => toArrowField(name, udt.sqlType, nullable, timeZoneId, largeVarTypes) + case g: GeometryType => + val fieldType = + new FieldType(nullable, ArrowType.Struct.INSTANCE, null) + + // WKB field is tagged with additional metadata so we can identify that the arrow + // struct actually represents a geometry schema. + val wkbFieldType = new FieldType( + false, + toArrowType(BinaryType, timeZoneId, largeVarTypes), + null, + Map("geometry" -> "true", "srid" -> g.srid.toString).asJava) + + new Field( + name, + fieldType, + Seq( + toArrowField("srid", IntegerType, false, timeZoneId, largeVarTypes), + new Field("wkb", wkbFieldType, Seq.empty[Field].asJava)).asJava) + + case g: GeographyType => + val fieldType = + new FieldType(nullable, ArrowType.Struct.INSTANCE, null, null) + + // WKB field is tagged with additional metadata so we can identify that the arrow + // struct actually represents a geography schema. + val wkbFieldType = new FieldType( + false, + toArrowType(BinaryType, timeZoneId, largeVarTypes), + null, + Map("geography" -> "true", "srid" -> g.srid.toString).asJava) + + new Field( + name, + fieldType, + Seq( + toArrowField("srid", IntegerType, false, timeZoneId, largeVarTypes), + new Field("wkb", wkbFieldType, Seq.empty[Field].asJava)).asJava) case _: VariantType => val fieldType = new FieldType(nullable, ArrowType.Struct.INSTANCE, null) // The metadata field is tagged with additional metadata so we can identify that the arrow @@ -175,6 +212,26 @@ private[sql] object ArrowUtils { } } + def isGeometryField(field: Field): Boolean = { + assert(field.getType.isInstanceOf[ArrowType.Struct]) + field.getChildren.asScala + .map(_.getName) + .asJava + .containsAll(Seq("wkb", "srid").asJava) && field.getChildren.asScala.exists { child => + child.getName == "wkb" && child.getMetadata.getOrDefault("geometry", "false") == "true" + } + } + + def isGeographyField(field: Field): Boolean = { + assert(field.getType.isInstanceOf[ArrowType.Struct]) + field.getChildren.asScala + .map(_.getName) + .asJava + .containsAll(Seq("wkb", "srid").asJava) && field.getChildren.asScala.exists { child => + child.getName == "wkb" && child.getMetadata.getOrDefault("geography", "false") == "true" + } + } + def fromArrowField(field: Field): DataType = { field.getType match { case _: ArrowType.Map => @@ -188,6 +245,26 @@ private[sql] object ArrowUtils { ArrayType(elementType, containsNull = elementField.isNullable) case ArrowType.Struct.INSTANCE if isVariantField(field) => VariantType + case ArrowType.Struct.INSTANCE if isGeometryField(field) => + // We expect that type metadata is associated with wkb field. + val metadataField = + field.getChildren.asScala.filter { child => child.getName == "wkb" }.head + val srid = metadataField.getMetadata.get("srid").toInt + if (srid == GeometryType.MIXED_SRID) { + GeometryType("ANY") + } else { + GeometryType(srid) + } + case ArrowType.Struct.INSTANCE if isGeographyField(field) => + // We expect that type metadata is associated with wkb field. + val metadataField = + field.getChildren.asScala.filter { child => child.getName == "wkb" }.head + val srid = metadataField.getMetadata.get("srid").toInt + if (srid == GeographyType.MIXED_SRID) { + GeographyType("ANY") + } else { + GeographyType(srid) + } case ArrowType.Struct.INSTANCE => val fields = field.getChildren().asScala.map { child => val dt = fromArrowField(child) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java index 3cf4b84ac0330..0a9942c4cf557 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/util/STUtils.java @@ -111,6 +111,10 @@ public static GeometryVal stGeomFromWKB(byte[] wkb) { return toPhysVal(Geometry.fromWkb(wkb)); } + public static GeometryVal stGeomFromWKB(byte[] wkb, int srid) { + return toPhysVal(Geometry.fromWkb(wkb, srid)); + } + // ST_SetSrid public static GeographyVal stSetSrid(GeographyVal geo, int srid) { // We only allow setting the SRID to geographic values. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 66116d7c952fd..019bc258579a8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -25,9 +25,12 @@ import org.apache.spark.SparkUnsupportedOperationException; import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.sql.catalyst.util.STUtils; import org.apache.spark.sql.util.ArrowUtils; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.GeographyVal; +import org.apache.spark.unsafe.types.GeometryVal; import org.apache.spark.unsafe.types.UTF8String; /** @@ -146,6 +149,30 @@ public ColumnarMap getMap(int rowId) { super(type); } + @Override + public GeographyVal getGeography(int rowId) { + if (isNullAt(rowId)) return null; + + GeographyType gt = (GeographyType) this.type; + int srid = getChild(0).getInt(rowId); + byte[] bytes = getChild(1).getBinary(rowId); + gt.assertSridAllowedForType(srid); + // TODO(GEO-602): Geog still does not support different SRIDs, once it does, + // we need to update this. + return (bytes == null) ? null : STUtils.stGeogFromWKB(bytes); + } + + @Override + public GeometryVal getGeometry(int rowId) { + if (isNullAt(rowId)) return null; + + GeometryType gt = (GeometryType) this.type; + int srid = getChild(0).getInt(rowId); + byte[] bytes = getChild(1).getBinary(rowId); + gt.assertSridAllowedForType(srid); + return (bytes == null) ? null : STUtils.stGeomFromWKB(bytes, srid); + } + public ArrowColumnVector(ValueVector vector) { this(ArrowUtils.fromArrowField(vector.getField())); initAccessor(vector); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala index 275fecebdafb8..8d68e74dbf874 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala @@ -24,6 +24,7 @@ import org.apache.arrow.vector.complex._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.catalyst.util.STUtils import org.apache.spark.sql.errors.ExecutionErrors import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils @@ -92,6 +93,16 @@ object ArrowWriter { createFieldWriter(vector.getChildByOrdinal(ordinal)) } new StructWriter(vector, children.toArray) + case (dt: GeometryType, vector: StructVector) => + val children = (0 until vector.size()).map { ordinal => + createFieldWriter(vector.getChildByOrdinal(ordinal)) + } + new GeometryWriter(dt, vector, children.toArray) + case (dt: GeographyType, vector: StructVector) => + val children = (0 until vector.size()).map { ordinal => + createFieldWriter(vector.getChildByOrdinal(ordinal)) + } + new GeographyWriter(dt, vector, children.toArray) case (dt, _) => throw ExecutionErrors.unsupportedDataTypeError(dt) } @@ -446,6 +457,42 @@ private[arrow] class StructWriter( } } +private[arrow] class GeographyWriter( + dt: GeographyType, + valueVector: StructVector, + children: Array[ArrowFieldWriter]) extends StructWriter(valueVector, children) { + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setIndexDefined(count) + + val geom = STUtils.deserializeGeog(input.getGeography(ordinal), dt) + val bytes = geom.getBytes + val srid = geom.getSrid + + val row = InternalRow(srid, bytes) + children(0).write(row, 0) + children(1).write(row, 1) + } +} + +private[arrow] class GeometryWriter( + dt: GeometryType, + valueVector: StructVector, + children: Array[ArrowFieldWriter]) extends StructWriter(valueVector, children) { + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setIndexDefined(count) + + val geom = STUtils.deserializeGeom(input.getGeometry(ordinal), dt) + val bytes = geom.getBytes + val srid = geom.getSrid + + val row = InternalRow(srid, bytes) + children(0).write(row, 0) + children(1).write(row, 1) + } +} + private[arrow] class MapWriter( val valueVector: MapVector, val structVector: StructVector, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala index 7124c94b390d0..8011e69e724c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala @@ -49,6 +49,10 @@ class ArrowUtilsSuite extends SparkFunSuite { roundtrip(BinaryType) roundtrip(DecimalType.SYSTEM_DEFAULT) roundtrip(DateType) + roundtrip(GeometryType("ANY")) + roundtrip(GeometryType(4326)) + roundtrip(GeographyType("ANY")) + roundtrip(GeographyType(4326)) roundtrip(YearMonthIntervalType()) roundtrip(DayTimeIntervalType()) checkError( diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala index b29d73be359b5..bc840df5c3fac 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala @@ -46,7 +46,7 @@ import org.apache.spark.sql.connect.client.arrow.FooEnum.FooEnum import org.apache.spark.sql.connect.test.ConnectFunSuite import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType} import org.apache.spark.unsafe.types.VariantVal -import org.apache.spark.util.SparkStringUtils +import org.apache.spark.util.{MaybeNull, SparkStringUtils} /** * Tests for encoding external data to and from arrow. @@ -218,20 +218,6 @@ class ArrowEncoderSuite extends ConnectFunSuite with BeforeAndAfterAll { } } - private case class MaybeNull(interval: Int) { - assert(interval > 1) - private var invocations = 0 - def apply[T](value: T): T = { - val result = if (invocations % interval == 0) { - null.asInstanceOf[T] - } else { - value - } - invocations += 1 - result - } - } - private def javaBigDecimal(i: Int): java.math.BigDecimal = { javaBigDecimal(i, DecimalType.DEFAULT_SCALE) } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index cf7494212cd55..b3e0f947296f7 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -49,6 +49,13 @@ spark-sketch_${scala.binary.version} ${project.version} + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${project.version} + tests + test + org.apache.spark spark-core_${scala.binary.version} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala index 99d245529e96d..2c0c0494bbacf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala @@ -17,16 +17,23 @@ package org.apache.spark.sql.execution.arrow +import scala.jdk.CollectionConverters._ + import org.apache.arrow.vector.VectorSchemaRoot import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row import org.apache.spark.sql.YearUDT import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.encoders.RowEncoder.{encoderFor => toRowEncoder} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.{Geography => InternalGeography, Geometry => InternalGeometry} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.{CalendarInterval, GeographyVal, GeometryVal, UTF8String} +import org.apache.spark.util.MaybeNull class ArrowWriterSuite extends SparkFunSuite { @@ -52,8 +59,16 @@ class ArrowWriterSuite extends SparkFunSuite { } writer.finish() + val dataModified = data.map { datum => + dt match { + case _: GeometryType => datum.asInstanceOf[GeometryVal].getBytes + case _: GeographyType => datum.asInstanceOf[GeographyVal].getBytes + case _ => datum + } + } + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) - data.zipWithIndex.foreach { + dataModified.zipWithIndex.foreach { case (null, rowId) => assert(reader.isNullAt(rowId)) case (datum, rowId) => val value = datatype match { @@ -74,12 +89,31 @@ class ArrowWriterSuite extends SparkFunSuite { case _: YearMonthIntervalType => reader.getInt(rowId) case _: DayTimeIntervalType => reader.getLong(rowId) case CalendarIntervalType => reader.getInterval(rowId) + case _: GeometryType => reader.getGeometry(rowId).getBytes + case _: GeographyType => reader.getGeography(rowId).getBytes } assert(value === datum) } writer.root.close() } + + val wkbs = Seq("010100000000000000000031400000000000001c40", + "010100000000000000000034400000000000003540") + .map { x => + x.grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + } + + val geographies = wkbs.map(x => InternalGeography.fromWkb(x, 4326).getValue) + val geometries = wkbs.map(x => InternalGeometry.fromWkb(x, 0).getValue) + val mixedGeometries = wkbs.zip(Seq(0, 4326)).map { + case (g, srid) => InternalGeometry.fromWkb(g, srid).getValue + } + + check(GeometryType(0), geometries) + check(GeographyType(4326), geographies) + check(GeometryType("ANY"), mixedGeometries) + check(GeographyType("ANY"), geographies) check(BooleanType, Seq(true, null, false)) check(ByteType, Seq(1.toByte, 2.toByte, null, 4.toByte)) check(ShortType, Seq(1.toShort, 2.toShort, null, 4.toShort)) @@ -110,6 +144,245 @@ class ArrowWriterSuite extends SparkFunSuite { check(new YearUDT, Seq(2020, 2021, null, 2022)) } + test("nested geographies") { + def check( + dt: StructType, + data: Seq[InternalRow]): Unit = { + val writer = ArrowWriter.create(dt.asInstanceOf[StructType], "UTC") + + // Write data to arrow. + data.toSeq.foreach { datum => + writer.write(datum) + } + writer.finish() + + // Create arrow vector readers. + val vectors = writer.root.getFieldVectors.asScala + .map { new ArrowColumnVector(_) }.toArray.asInstanceOf[Array[ColumnVector]] + + val batch = new ColumnarBatch(vectors, writer.root.getRowCount.toInt) + + data.zipWithIndex.foreach { case (datum, i) => + // Read data from arrow. + val internalRow = batch.getRow(i) + + // All nullable results first must check whether the value is null. + if (datum.getStruct(0, 4) == null || internalRow.getStruct(0, 4) == null) { + assert(datum.getStruct(0, 4) == null && internalRow.getStruct(0, 4) == null) + } else { + val expectedStruct = datum.getStruct(0, 4) + val actualStruct = internalRow.getStruct(0, 4) + assert(expectedStruct.getInt(0) === actualStruct.getInt(0)) + assert(expectedStruct.getInt(2) === actualStruct.getInt(2)) + + if (expectedStruct.getGeography(1) == null || + actualStruct.getGeography(1) == null) { + assert(expectedStruct.getGeography(1) == null && actualStruct.getGeography(1) == null) + } else { + assert(expectedStruct.getGeography(1).getBytes === + actualStruct.getGeography(1).getBytes) + } + if (expectedStruct.getGeography(3) == null || + actualStruct.getGeography(3) == null) { + assert(expectedStruct.getGeography(3) == null && actualStruct.getGeography(3) == null) + } else { + assert(expectedStruct.getGeography(3).getBytes === + actualStruct.getGeography(3).getBytes) + } + + if (datum.getArray(1) == null || + internalRow.getArray(1) == null) { + assert(internalRow.getArray(1) == null && datum.getArray(1) == null) + } else { + internalRow.getArray(1).toSeq[GeographyVal](GeographyType(4326)) + .zip(datum.getArray(1).toSeq[GeographyVal](GeographyType(4326))).foreach { + case (actual, expected) => + assert(actual.getBytes === expected.getBytes) + } + } + + if (datum.getMap(2) == null || + internalRow.getMap(2) == null) { + assert(internalRow.getMap(2) == null && datum.getMap(2) == null) + } else { + assert(internalRow.getMap(2).keyArray().toSeq(StringType) === + datum.getMap(2).keyArray().toSeq(StringType)) + internalRow.getMap(2).valueArray().toSeq[GeographyVal](GeographyType("ANY")) + .zip(datum.getMap(2).valueArray().toSeq[GeographyVal](GeographyType("ANY"))).foreach { + case (actual, expected) => + assert((actual == null && expected == null) || + actual.getBytes === expected.getBytes) + } + } + } + } + + writer.root.close() + } + + val point1 = "010100000000000000000031400000000000001C40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val point2 = "010100000000000000000035400000000000001E40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + + val schema = new StructType() + .add( + "s", + new StructType() + .add("i1", "int") + .add("g0", "geography(4326)") + .add("i2", "int") + .add("g1", "geography(4326)")) + .add("a", "array") + .add("m", "map") + + val maybeNull5 = MaybeNull(5) + val maybeNull7 = MaybeNull(7) + val maybeNull11 = MaybeNull(11) + val maybeNull13 = MaybeNull(13) + val maybeNull17 = MaybeNull(17) + + val nestedGeographySerializer = ExpressionEncoder(toRowEncoder(schema)).createSerializer() + val data = Iterator + .tabulate(100)(i => + nestedGeographySerializer.apply( + (Row( + maybeNull5( + Row( + i, + maybeNull7(org.apache.spark.sql.types.Geography.fromWKB(point1)), + i + 1, + maybeNull11(org.apache.spark.sql.types.Geography.fromWKB(point2, 4326)))), + maybeNull7((0 until 10).map(j => + org.apache.spark.sql.types.Geography.fromWKB(point2, 4326))), + maybeNull13( + Map((i.toString, maybeNull17( + org.apache.spark.sql.types.Geography.fromWKB(point1, 4326))))))))) + .map(_.copy()).toSeq + + check(schema, data) + } + + test("nested geometries") { + def check( + dt: StructType, + data: Seq[InternalRow]): Unit = { + val writer = ArrowWriter.create(dt.asInstanceOf[StructType], "UTC") + + // Write data to arrow. + data.toSeq.foreach { datum => + writer.write(datum) + } + writer.finish() + + // Create arrow vector readers. + val vectors = writer.root.getFieldVectors.asScala + .map { new ArrowColumnVector(_) }.toArray.asInstanceOf[Array[ColumnVector]] + + val batch = new ColumnarBatch(vectors, writer.root.getRowCount.toInt) + data.zipWithIndex.foreach { case (datum, i) => + // Read data from arrow. + val internalRow = batch.getRow(i) + + // All nullable results first must check whether the value is null. + if (datum.getStruct(0, 4) == null || internalRow.getStruct(0, 4) == null) { + assert(datum.getStruct(0, 4) == null && internalRow.getStruct(0, 4) == null) + } else { + val expectedStruct = datum.getStruct(0, 4) + val actualStruct = internalRow.getStruct(0, 4) + assert(expectedStruct.getInt(0) === actualStruct.getInt(0)) + assert(expectedStruct.getInt(2) === actualStruct.getInt(2)) + + if (expectedStruct.getGeometry(1) == null || + actualStruct.getGeometry(1) == null) { + assert(expectedStruct.getGeometry(1) == null && actualStruct.getGeometry(1) == null) + } else { + assert(expectedStruct.getGeometry(1).getBytes === + actualStruct.getGeometry(1).getBytes) + } + if (expectedStruct.getGeometry(3) == null || + actualStruct.getGeometry(3) == null) { + assert(expectedStruct.getGeometry(3) == null && actualStruct.getGeometry(3) == null) + } else { + assert(expectedStruct.getGeometry(3).getBytes === + actualStruct.getGeometry(3).getBytes) + } + + if (datum.getArray(1) == null || + internalRow.getArray(1) == null) { + assert(internalRow.getArray(1) == null && datum.getArray(1) == null) + } else { + internalRow.getArray(1).toSeq[GeometryVal](GeometryType(0)) + .zip(datum.getArray(1).toSeq[GeometryVal](GeometryType(0))).foreach { + case (actual, expected) => + assert(actual.getBytes === expected.getBytes) + } + } + + if (datum.getMap(2) == null || + internalRow.getMap(2) == null) { + assert(internalRow.getMap(2) == null && datum.getMap(2) == null) + } else { + assert(internalRow.getMap(2).keyArray().toSeq(StringType) === + datum.getMap(2).keyArray().toSeq(StringType)) + internalRow.getMap(2).valueArray().toSeq[GeometryVal](GeometryType("ANY")) + .zip(datum.getMap(2).valueArray().toSeq[GeometryVal](GeometryType("ANY"))).foreach { + case (actual, expected) => + assert((actual == null && expected == null) || + actual.getBytes === expected.getBytes) + } + } + } + } + + writer.root.close() + } + + val point1 = "010100000000000000000031400000000000001C40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val point2 = "010100000000000000000035400000000000001E40" + .grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + + val schema = new StructType() + .add( + "s", + new StructType() + .add("i1", "int") + .add("g0", "geometry(0)") + .add("i2", "int") + .add("g4326", "geometry(4326)")) + .add("a", "array") + .add("m", "map") + + val maybeNull5 = MaybeNull(5) + val maybeNull7 = MaybeNull(7) + val maybeNull11 = MaybeNull(11) + val maybeNull13 = MaybeNull(13) + val maybeNull17 = MaybeNull(17) + + val nestedGeometrySerializer = ExpressionEncoder(toRowEncoder(schema)).createSerializer() + val data = Iterator + .tabulate(100) { i => + val mixedSrid = if (i % 2 == 0) 0 else 4326 + + nestedGeometrySerializer.apply( + (Row( + maybeNull5( + Row( + i, + maybeNull7(org.apache.spark.sql.types.Geometry.fromWKB(point1, 0)), + i + 1, + maybeNull11(org.apache.spark.sql.types.Geometry.fromWKB(point2, 4326)))), + maybeNull7((0 until 10).map(j => + org.apache.spark.sql.types.Geometry.fromWKB(point2, 0))), + maybeNull13( + Map((i.toString, maybeNull17( + org.apache.spark.sql.types.Geometry.fromWKB(point1, mixedSrid)))))))) + }.map(_.copy()).toSeq + + check(schema, data) + } + test("get multiple") { def check(dt: DataType, data: Seq[Any], timeZoneId: String = null): Unit = { val datatype = dt match { From b902cf350496d8d75cee93059432aba41d471a5f Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 6 Nov 2025 10:45:08 -0800 Subject: [PATCH 055/400] [SPARK-52463][SDP] Add support for cluster_by in Python Pipelines APIs ### What changes were proposed in this pull request? In the `table` and `materialized_view` decorators, accept a `cluster_by` argument that determines the clustering columns. ### Why are the changes needed? Parity with the `clusterBy` argument accepted by `DataStreamReader` and `DataFrameWriter`. ### Does this PR introduce _any_ user-facing change? Adds a new parameter to public APIs. ### How was this patch tested? Unit tests and integration tests. ### Was this patch authored or co-authored using generative AI tooling? Closes #52831 from sryza/cluster-by. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit a927a14a59e2d07ae10e73f3e28f3ca1b1208929) Signed-off-by: Dongjoon Hyun --- python/pyspark/pipelines/api.py | 18 +- python/pyspark/pipelines/output.py | 2 + .../spark_connect_graph_element_registry.py | 1 + .../sql/connect/proto/pipelines_pb2.py | 88 +++---- .../sql/connect/proto/pipelines_pb2.pyi | 9 + .../protobuf/spark/connect/pipelines.proto | 3 + .../connect/pipelines/PipelinesHandler.scala | 2 + .../pipelines/PythonPipelineSuite.scala | 37 +++ .../pipelines/TestPipelineDefinition.scala | 2 + .../sql/pipelines/graph/DatasetManager.scala | 27 +- .../sql/pipelines/graph/FlowExecution.scala | 16 +- .../graph/SqlGraphRegistrationContext.scala | 3 + .../spark/sql/pipelines/graph/elements.scala | 2 + .../graph/MaterializeTablesSuite.scala | 246 +++++++++++++++++- .../spark/sql/pipelines/utils/APITest.scala | 60 +++++ .../utils/TestGraphRegistrationContext.scala | 6 + 16 files changed, 464 insertions(+), 58 deletions(-) diff --git a/python/pyspark/pipelines/api.py b/python/pyspark/pipelines/api.py index b68cc30b43a7d..c01e8524eee20 100644 --- a/python/pyspark/pipelines/api.py +++ b/python/pyspark/pipelines/api.py @@ -76,6 +76,7 @@ def _validate_stored_dataset_args( name: Optional[str], table_properties: Optional[Dict[str, str]], partition_cols: Optional[List[str]], + cluster_by: Optional[List[str]], ) -> None: if name is not None and type(name) is not str: raise PySparkTypeError( @@ -91,6 +92,7 @@ def _validate_stored_dataset_args( }, ) validate_optional_list_of_str_arg(arg_name="partition_cols", arg_value=partition_cols) + validate_optional_list_of_str_arg(arg_name="cluster_by", arg_value=cluster_by) @overload @@ -107,6 +109,7 @@ def table( spark_conf: Optional[Dict[str, str]] = None, table_properties: Optional[Dict[str, str]] = None, partition_cols: Optional[List[str]] = None, + cluster_by: Optional[List[str]] = None, schema: Optional[Union[StructType, str]] = None, ) -> Callable[[QueryFunction], None]: ... @@ -120,6 +123,7 @@ def table( spark_conf: Optional[Dict[str, str]] = None, table_properties: Optional[Dict[str, str]] = None, partition_cols: Optional[List[str]] = None, + cluster_by: Optional[List[str]] = None, schema: Optional[Union[StructType, str]] = None, format: Optional[str] = None, ) -> Union[Callable[[QueryFunction], None], None]: @@ -142,11 +146,12 @@ def table( :param table_properties: A dict where the keys are the property names and the values are the \ property values. These properties will be set on the table. :param partition_cols: A list containing the column names of the partition columns. + :param cluster_by: A list containing the column names of the cluster columns. :param schema: Explicit Spark SQL schema to materialize this table with. Supports either a \ Pyspark StructType or a SQL DDL string, such as "a INT, b STRING". :param format: The format of the table, e.g. "parquet". """ - _validate_stored_dataset_args(name, table_properties, partition_cols) + _validate_stored_dataset_args(name, table_properties, partition_cols, cluster_by) source_code_location = get_caller_source_code_location(stacklevel=1) @@ -163,6 +168,7 @@ def outer( name=resolved_name, table_properties=table_properties or {}, partition_cols=partition_cols, + cluster_by=cluster_by, schema=schema, source_code_location=source_code_location, format=format, @@ -209,6 +215,7 @@ def materialized_view( spark_conf: Optional[Dict[str, str]] = None, table_properties: Optional[Dict[str, str]] = None, partition_cols: Optional[List[str]] = None, + cluster_by: Optional[List[str]] = None, schema: Optional[Union[StructType, str]] = None, ) -> Callable[[QueryFunction], None]: ... @@ -222,6 +229,7 @@ def materialized_view( spark_conf: Optional[Dict[str, str]] = None, table_properties: Optional[Dict[str, str]] = None, partition_cols: Optional[List[str]] = None, + cluster_by: Optional[List[str]] = None, schema: Optional[Union[StructType, str]] = None, format: Optional[str] = None, ) -> Union[Callable[[QueryFunction], None], None]: @@ -244,11 +252,12 @@ def materialized_view( :param table_properties: A dict where the keys are the property names and the values are the \ property values. These properties will be set on the table. :param partition_cols: A list containing the column names of the partition columns. + :param cluster_by: A list containing the column names of the cluster columns. :param schema: Explicit Spark SQL schema to materialize this table with. Supports either a \ Pyspark StructType or a SQL DDL string, such as "a INT, b STRING". :param format: The format of the table, e.g. "parquet". """ - _validate_stored_dataset_args(name, table_properties, partition_cols) + _validate_stored_dataset_args(name, table_properties, partition_cols, cluster_by) source_code_location = get_caller_source_code_location(stacklevel=1) @@ -265,6 +274,7 @@ def outer( name=resolved_name, table_properties=table_properties or {}, partition_cols=partition_cols, + cluster_by=cluster_by, schema=schema, source_code_location=source_code_location, format=format, @@ -403,6 +413,7 @@ def create_streaming_table( comment: Optional[str] = None, table_properties: Optional[Dict[str, str]] = None, partition_cols: Optional[List[str]] = None, + cluster_by: Optional[List[str]] = None, schema: Optional[Union[StructType, str]] = None, format: Optional[str] = None, ) -> None: @@ -417,6 +428,7 @@ def create_streaming_table( :param table_properties: A dict where the keys are the property names and the values are the \ property values. These properties will be set on the table. :param partition_cols: A list containing the column names of the partition columns. + :param cluster_by: A list containing the column names of the cluster columns. :param schema Explicit Spark SQL schema to materialize this table with. Supports either a \ Pyspark StructType or a SQL DDL string, such as "a INT, b STRING". :param format: The format of the table, e.g. "parquet". @@ -435,6 +447,7 @@ def create_streaming_table( }, ) validate_optional_list_of_str_arg(arg_name="partition_cols", arg_value=partition_cols) + validate_optional_list_of_str_arg(arg_name="cluster_by", arg_value=cluster_by) source_code_location = get_caller_source_code_location(stacklevel=1) @@ -444,6 +457,7 @@ def create_streaming_table( source_code_location=source_code_location, table_properties=table_properties or {}, partition_cols=partition_cols, + cluster_by=cluster_by, schema=schema, format=format, ) diff --git a/python/pyspark/pipelines/output.py b/python/pyspark/pipelines/output.py index 84e950f161742..92058e68721f4 100644 --- a/python/pyspark/pipelines/output.py +++ b/python/pyspark/pipelines/output.py @@ -45,6 +45,7 @@ class Table(Output): :param table_properties: A dict where the keys are the property names and the values are the property values. These properties will be set on the table. :param partition_cols: A list containing the column names of the partition columns. + :param cluster_by: A list containing the column names of the cluster columns. :param schema Explicit Spark SQL schema to materialize this table with. Supports either a Pyspark StructType or a SQL DDL string, such as "a INT, b STRING". :param format: The format of the table, e.g. "parquet". @@ -52,6 +53,7 @@ class Table(Output): table_properties: Mapping[str, str] partition_cols: Optional[Sequence[str]] + cluster_by: Optional[Sequence[str]] schema: Optional[Union[StructType, str]] format: Optional[str] diff --git a/python/pyspark/pipelines/spark_connect_graph_element_registry.py b/python/pyspark/pipelines/spark_connect_graph_element_registry.py index 5c5ef9fc30401..e8a8561c3e749 100644 --- a/python/pyspark/pipelines/spark_connect_graph_element_registry.py +++ b/python/pyspark/pipelines/spark_connect_graph_element_registry.py @@ -63,6 +63,7 @@ def register_output(self, output: Output) -> None: table_details = pb2.PipelineCommand.DefineOutput.TableDetails( table_properties=output.table_properties, partition_cols=output.partition_cols, + clustering_columns=output.cluster_by, format=output.format, # Even though schema_string is not required, the generated Python code seems to # erroneously think it is required. diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py index d7321fa7cf0c1..139de83dc1aaf 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.py +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py @@ -42,7 +42,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\x9c"\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12R\n\rdefine_output\x18\x02 \x01(\x0b\x32+.spark.connect.PipelineCommand.DefineOutputH\x00R\x0c\x64\x65\x66ineOutput\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x12\xa1\x01\n*get_query_function_execution_signal_stream\x18\x07 \x01(\x0b\x32\x44.spark.connect.PipelineCommand.GetQueryFunctionExecutionSignalStreamH\x00R%getQueryFunctionExecutionSignalStream\x12\x88\x01\n!define_flow_query_function_result\x18\x08 \x01(\x0b\x32<.spark.connect.PipelineCommand.DefineFlowQueryFunctionResultH\x00R\x1d\x64\x65\x66ineFlowQueryFunctionResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xb4\x02\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\xe3\t\n\x0c\x44\x65\x66ineOutput\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12$\n\x0boutput_name\x18\x02 \x01(\tH\x02R\noutputName\x88\x01\x01\x12?\n\x0boutput_type\x18\x03 \x01(\x0e\x32\x19.spark.connect.OutputTypeH\x03R\noutputType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x04R\x07\x63omment\x88\x01\x01\x12X\n\x14source_code_location\x18\x05 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12_\n\rtable_details\x18\x06 \x01(\x0b\x32\x38.spark.connect.PipelineCommand.DefineOutput.TableDetailsH\x00R\x0ctableDetails\x12\\\n\x0csink_details\x18\x07 \x01(\x0b\x32\x37.spark.connect.PipelineCommand.DefineOutput.SinkDetailsH\x00R\x0bsinkDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x91\x03\n\x0cTableDetails\x12x\n\x10table_properties\x18\x01 \x03(\x0b\x32M.spark.connect.PipelineCommand.DefineOutput.TableDetails.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x02 \x03(\tR\rpartitionCols\x12\x1b\n\x06\x66ormat\x18\x03 \x01(\tH\x01R\x06\x66ormat\x88\x01\x01\x12\x43\n\x10schema_data_type\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x0eschemaDataType\x12%\n\rschema_string\x18\x05 \x01(\tH\x00R\x0cschemaString\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x08\n\x06schemaB\t\n\x07_format\x1a\xd1\x01\n\x0bSinkDetails\x12^\n\x07options\x18\x01 \x03(\x0b\x32\x44.spark.connect.PipelineCommand.DefineOutput.SinkDetails.OptionsEntryR\x07options\x12\x1b\n\x06\x66ormat\x18\x02 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0e\n\x0c_output_nameB\x0e\n\x0c_output_typeB\n\n\x08_commentB\x17\n\x15_source_code_location\x1a\xdd\x06\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x03R\x11targetDatasetName\x88\x01\x01\x12Q\n\x08sql_conf\x18\x04 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12 \n\tclient_id\x18\x05 \x01(\tH\x04R\x08\x63lientId\x88\x01\x01\x12X\n\x14source_code_location\x18\x06 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12x\n\x15relation_flow_details\x18\x07 \x01(\x0b\x32\x42.spark.connect.PipelineCommand.DefineFlow.WriteRelationFlowDetailsH\x00R\x13relationFlowDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x61\n\x18WriteRelationFlowDetails\x12\x38\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x08relation\x88\x01\x01\x42\x0b\n\t_relation\x1a:\n\x08Response\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x42\x0c\n\n_flow_nameB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0c\n\n_client_idB\x17\n\x15_source_code_location\x1a\xc2\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x12\x1d\n\x07storage\x18\x06 \x01(\tH\x03R\x07storage\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dryB\n\n\x08_storage\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_text\x1a\x9e\x01\n%GetQueryFunctionExecutionSignalStream\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tclient_id\x18\x02 \x01(\tH\x01R\x08\x63lientId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_client_id\x1a\xdd\x01\n\x1d\x44\x65\x66ineFlowQueryFunctionResult\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x12/\n\x11\x64\x61taflow_graph_id\x18\x02 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x38\n\x08relation\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x02R\x08relation\x88\x01\x01\x42\x0c\n\n_flow_nameB\x14\n\x12_dataflow_graph_idB\x0b\n\t_relationB\x0e\n\x0c\x63ommand_type"\xf0\x05\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x12k\n\x14\x64\x65\x66ine_output_result\x18\x02 \x01(\x0b\x32\x37.spark.connect.PipelineCommandResult.DefineOutputResultH\x00R\x12\x64\x65\x66ineOutputResult\x12\x65\n\x12\x64\x65\x66ine_flow_result\x18\x03 \x01(\x0b\x32\x35.spark.connect.PipelineCommandResult.DefineFlowResultH\x00R\x10\x64\x65\x66ineFlowResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x85\x01\n\x12\x44\x65\x66ineOutputResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifier\x1a\x83\x01\n\x10\x44\x65\x66ineFlowResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifierB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message"\xf1\x01\n\x12SourceCodeLocation\x12 \n\tfile_name\x18\x01 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12$\n\x0bline_number\x18\x02 \x01(\x05H\x01R\nlineNumber\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x03 \x01(\tH\x02R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x0c\n\n_file_nameB\x0e\n\x0c_line_numberB\x12\n\x10_definition_path"E\n$PipelineQueryFunctionExecutionSignal\x12\x1d\n\nflow_names\x18\x01 \x03(\tR\tflowNames"\xd7\x01\n\x17PipelineAnalysisContext\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x02 \x01(\tH\x01R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x14\n\x12_dataflow_graph_idB\x12\n\x10_definition_path*i\n\nOutputType\x12\x1b\n\x17OUTPUT_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x12\x08\n\x04SINK\x10\x04\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\xcb"\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12R\n\rdefine_output\x18\x02 \x01(\x0b\x32+.spark.connect.PipelineCommand.DefineOutputH\x00R\x0c\x64\x65\x66ineOutput\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x12\xa1\x01\n*get_query_function_execution_signal_stream\x18\x07 \x01(\x0b\x32\x44.spark.connect.PipelineCommand.GetQueryFunctionExecutionSignalStreamH\x00R%getQueryFunctionExecutionSignalStream\x12\x88\x01\n!define_flow_query_function_result\x18\x08 \x01(\x0b\x32<.spark.connect.PipelineCommand.DefineFlowQueryFunctionResultH\x00R\x1d\x64\x65\x66ineFlowQueryFunctionResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xb4\x02\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x92\n\n\x0c\x44\x65\x66ineOutput\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12$\n\x0boutput_name\x18\x02 \x01(\tH\x02R\noutputName\x88\x01\x01\x12?\n\x0boutput_type\x18\x03 \x01(\x0e\x32\x19.spark.connect.OutputTypeH\x03R\noutputType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x04R\x07\x63omment\x88\x01\x01\x12X\n\x14source_code_location\x18\x05 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12_\n\rtable_details\x18\x06 \x01(\x0b\x32\x38.spark.connect.PipelineCommand.DefineOutput.TableDetailsH\x00R\x0ctableDetails\x12\\\n\x0csink_details\x18\x07 \x01(\x0b\x32\x37.spark.connect.PipelineCommand.DefineOutput.SinkDetailsH\x00R\x0bsinkDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xc0\x03\n\x0cTableDetails\x12x\n\x10table_properties\x18\x01 \x03(\x0b\x32M.spark.connect.PipelineCommand.DefineOutput.TableDetails.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x02 \x03(\tR\rpartitionCols\x12\x1b\n\x06\x66ormat\x18\x03 \x01(\tH\x01R\x06\x66ormat\x88\x01\x01\x12\x43\n\x10schema_data_type\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x0eschemaDataType\x12%\n\rschema_string\x18\x05 \x01(\tH\x00R\x0cschemaString\x12-\n\x12\x63lustering_columns\x18\x06 \x03(\tR\x11\x63lusteringColumns\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x08\n\x06schemaB\t\n\x07_format\x1a\xd1\x01\n\x0bSinkDetails\x12^\n\x07options\x18\x01 \x03(\x0b\x32\x44.spark.connect.PipelineCommand.DefineOutput.SinkDetails.OptionsEntryR\x07options\x12\x1b\n\x06\x66ormat\x18\x02 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0e\n\x0c_output_nameB\x0e\n\x0c_output_typeB\n\n\x08_commentB\x17\n\x15_source_code_location\x1a\xdd\x06\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x03R\x11targetDatasetName\x88\x01\x01\x12Q\n\x08sql_conf\x18\x04 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12 \n\tclient_id\x18\x05 \x01(\tH\x04R\x08\x63lientId\x88\x01\x01\x12X\n\x14source_code_location\x18\x06 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12x\n\x15relation_flow_details\x18\x07 \x01(\x0b\x32\x42.spark.connect.PipelineCommand.DefineFlow.WriteRelationFlowDetailsH\x00R\x13relationFlowDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x61\n\x18WriteRelationFlowDetails\x12\x38\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x08relation\x88\x01\x01\x42\x0b\n\t_relation\x1a:\n\x08Response\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x42\x0c\n\n_flow_nameB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0c\n\n_client_idB\x17\n\x15_source_code_location\x1a\xc2\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x12\x1d\n\x07storage\x18\x06 \x01(\tH\x03R\x07storage\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dryB\n\n\x08_storage\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_text\x1a\x9e\x01\n%GetQueryFunctionExecutionSignalStream\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tclient_id\x18\x02 \x01(\tH\x01R\x08\x63lientId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_client_id\x1a\xdd\x01\n\x1d\x44\x65\x66ineFlowQueryFunctionResult\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x12/\n\x11\x64\x61taflow_graph_id\x18\x02 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x38\n\x08relation\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x02R\x08relation\x88\x01\x01\x42\x0c\n\n_flow_nameB\x14\n\x12_dataflow_graph_idB\x0b\n\t_relationB\x0e\n\x0c\x63ommand_type"\xf0\x05\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x12k\n\x14\x64\x65\x66ine_output_result\x18\x02 \x01(\x0b\x32\x37.spark.connect.PipelineCommandResult.DefineOutputResultH\x00R\x12\x64\x65\x66ineOutputResult\x12\x65\n\x12\x64\x65\x66ine_flow_result\x18\x03 \x01(\x0b\x32\x35.spark.connect.PipelineCommandResult.DefineFlowResultH\x00R\x10\x64\x65\x66ineFlowResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x85\x01\n\x12\x44\x65\x66ineOutputResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifier\x1a\x83\x01\n\x10\x44\x65\x66ineFlowResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifierB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message"\xf1\x01\n\x12SourceCodeLocation\x12 \n\tfile_name\x18\x01 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12$\n\x0bline_number\x18\x02 \x01(\x05H\x01R\nlineNumber\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x03 \x01(\tH\x02R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x0c\n\n_file_nameB\x0e\n\x0c_line_numberB\x12\n\x10_definition_path"E\n$PipelineQueryFunctionExecutionSignal\x12\x1d\n\nflow_names\x18\x01 \x03(\tR\tflowNames"\xd7\x01\n\x17PipelineAnalysisContext\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x02 \x01(\tH\x01R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x14\n\x12_dataflow_graph_idB\x12\n\x10_definition_path*i\n\nOutputType\x12\x1b\n\x17OUTPUT_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x12\x08\n\x04SINK\x10\x04\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -69,10 +69,10 @@ ]._serialized_options = b"8\001" _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._loaded_options = None _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_options = b"8\001" - _globals["_OUTPUTTYPE"]._serialized_start = 6058 - _globals["_OUTPUTTYPE"]._serialized_end = 6163 + _globals["_OUTPUTTYPE"]._serialized_start = 6105 + _globals["_OUTPUTTYPE"]._serialized_end = 6210 _globals["_PIPELINECOMMAND"]._serialized_start = 195 - _globals["_PIPELINECOMMAND"]._serialized_end = 4575 + _globals["_PIPELINECOMMAND"]._serialized_end = 4622 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_start = 1129 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_end = 1437 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH_SQLCONFENTRY"]._serialized_start = 1338 @@ -80,51 +80,51 @@ _globals["_PIPELINECOMMAND_DROPDATAFLOWGRAPH"]._serialized_start = 1439 _globals["_PIPELINECOMMAND_DROPDATAFLOWGRAPH"]._serialized_end = 1529 _globals["_PIPELINECOMMAND_DEFINEOUTPUT"]._serialized_start = 1532 - _globals["_PIPELINECOMMAND_DEFINEOUTPUT"]._serialized_end = 2783 + _globals["_PIPELINECOMMAND_DEFINEOUTPUT"]._serialized_end = 2830 _globals["_PIPELINECOMMAND_DEFINEOUTPUT_TABLEDETAILS"]._serialized_start = 2068 - _globals["_PIPELINECOMMAND_DEFINEOUTPUT_TABLEDETAILS"]._serialized_end = 2469 + _globals["_PIPELINECOMMAND_DEFINEOUTPUT_TABLEDETAILS"]._serialized_end = 2516 _globals[ "_PIPELINECOMMAND_DEFINEOUTPUT_TABLEDETAILS_TABLEPROPERTIESENTRY" - ]._serialized_start = 2382 + ]._serialized_start = 2429 _globals[ "_PIPELINECOMMAND_DEFINEOUTPUT_TABLEDETAILS_TABLEPROPERTIESENTRY" - ]._serialized_end = 2448 - _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS"]._serialized_start = 2472 - _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS"]._serialized_end = 2681 - _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS_OPTIONSENTRY"]._serialized_start = 2612 - _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS_OPTIONSENTRY"]._serialized_end = 2670 - _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_start = 2786 - _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_end = 3647 + ]._serialized_end = 2495 + _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS"]._serialized_start = 2519 + _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS"]._serialized_end = 2728 + _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS_OPTIONSENTRY"]._serialized_start = 2659 + _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS_OPTIONSENTRY"]._serialized_end = 2717 + _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_start = 2833 + _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_end = 3694 _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_start = 1338 _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_end = 1396 - _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_start = 3380 - _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_end = 3477 - _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_start = 3479 - _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_end = 3537 - _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_start = 3650 - _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_end = 3972 - _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_start = 3975 - _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_end = 4174 - _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_start = 4177 - _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_end = 4335 - _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_start = 4338 - _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_end = 4559 - _globals["_PIPELINECOMMANDRESULT"]._serialized_start = 4578 - _globals["_PIPELINECOMMANDRESULT"]._serialized_end = 5330 - _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_start = 4947 - _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_end = 5045 - _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_start = 5048 - _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_end = 5181 - _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_start = 5184 - _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_end = 5315 - _globals["_PIPELINEEVENTRESULT"]._serialized_start = 5332 - _globals["_PIPELINEEVENTRESULT"]._serialized_end = 5405 - _globals["_PIPELINEEVENT"]._serialized_start = 5407 - _globals["_PIPELINEEVENT"]._serialized_end = 5523 - _globals["_SOURCECODELOCATION"]._serialized_start = 5526 - _globals["_SOURCECODELOCATION"]._serialized_end = 5767 - _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_start = 5769 - _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_end = 5838 - _globals["_PIPELINEANALYSISCONTEXT"]._serialized_start = 5841 - _globals["_PIPELINEANALYSISCONTEXT"]._serialized_end = 6056 + _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_start = 3427 + _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_end = 3524 + _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_start = 3526 + _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_end = 3584 + _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_start = 3697 + _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_end = 4019 + _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_start = 4022 + _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_end = 4221 + _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_start = 4224 + _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_end = 4382 + _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_start = 4385 + _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_end = 4606 + _globals["_PIPELINECOMMANDRESULT"]._serialized_start = 4625 + _globals["_PIPELINECOMMANDRESULT"]._serialized_end = 5377 + _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_start = 4994 + _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_end = 5092 + _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_start = 5095 + _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_end = 5228 + _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_start = 5231 + _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_end = 5362 + _globals["_PIPELINEEVENTRESULT"]._serialized_start = 5379 + _globals["_PIPELINEEVENTRESULT"]._serialized_end = 5452 + _globals["_PIPELINEEVENT"]._serialized_start = 5454 + _globals["_PIPELINEEVENT"]._serialized_end = 5570 + _globals["_SOURCECODELOCATION"]._serialized_start = 5573 + _globals["_SOURCECODELOCATION"]._serialized_end = 5814 + _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_start = 5816 + _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_end = 5885 + _globals["_PIPELINEANALYSISCONTEXT"]._serialized_start = 5888 + _globals["_PIPELINEANALYSISCONTEXT"]._serialized_end = 6103 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi index b9170e763ed92..60d131037c99d 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi @@ -240,6 +240,7 @@ class PipelineCommand(google.protobuf.message.Message): FORMAT_FIELD_NUMBER: builtins.int SCHEMA_DATA_TYPE_FIELD_NUMBER: builtins.int SCHEMA_STRING_FIELD_NUMBER: builtins.int + CLUSTERING_COLUMNS_FIELD_NUMBER: builtins.int @property def table_properties( self, @@ -255,6 +256,11 @@ class PipelineCommand(google.protobuf.message.Message): @property def schema_data_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ... schema_string: builtins.str + @property + def clustering_columns( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """Optional cluster columns for the table.""" def __init__( self, *, @@ -263,6 +269,7 @@ class PipelineCommand(google.protobuf.message.Message): format: builtins.str | None = ..., schema_data_type: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., schema_string: builtins.str = ..., + clustering_columns: collections.abc.Iterable[builtins.str] | None = ..., ) -> None: ... def HasField( self, @@ -284,6 +291,8 @@ class PipelineCommand(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "_format", b"_format", + "clustering_columns", + b"clustering_columns", "format", b"format", "partition_cols", diff --git a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto index c6a5e571f9792..0fa36f8a15143 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto @@ -104,6 +104,9 @@ message PipelineCommand { spark.connect.DataType schema_data_type = 4; string schema_string = 5; } + + // Optional cluster columns for the table. + repeated string clustering_columns = 6; } // Metadata that's only applicable to sinks. diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala index 7e69e546893e6..0929b07be5237 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala @@ -203,6 +203,8 @@ private[connect] object PipelinesHandler extends Logging { }, partitionCols = Option(tableDetails.getPartitionColsList.asScala.toSeq) .filter(_.nonEmpty), + clusterCols = Option(tableDetails.getClusteringColumnsList.asScala.toSeq) + .filter(_.nonEmpty), properties = tableDetails.getTablePropertiesMap.asScala.toMap, origin = QueryOrigin( filePath = Option.when(output.getSourceCodeLocation.hasFileName)( diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala index 79c34ac46b9fb..1a72d112aa2ef 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala @@ -865,4 +865,41 @@ class PythonPipelineSuite (exitCode, output.toSeq) } + + test("empty cluster_by list should work and create table with no clustering") { + withTable("mv", "st") { + val graph = buildGraph(""" + |from pyspark.sql.functions import col + | + |@dp.materialized_view(cluster_by = []) + |def mv(): + | return spark.range(5).withColumn("id_mod", col("id") % 2) + | + |@dp.table(cluster_by = []) + |def st(): + | return spark.readStream.table("mv") + |""".stripMargin) + val updateContext = + new PipelineUpdateContextImpl(graph, eventCallback = _ => (), storageRoot = storageRoot) + updateContext.pipelineExecution.runPipeline() + updateContext.pipelineExecution.awaitCompletion() + + // Check tables are created with no clustering transforms + val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] + + val mvIdentifier = Identifier.of(Array("default"), "mv") + val mvTable = catalog.loadTable(mvIdentifier) + val mvTransforms = mvTable.partitioning() + assert( + mvTransforms.isEmpty, + s"MaterializedView should have no transforms, but got: ${mvTransforms.mkString(", ")}") + + val stIdentifier = Identifier.of(Array("default"), "st") + val stTable = catalog.loadTable(stIdentifier) + val stTransforms = stTable.partitioning() + assert( + stTransforms.isEmpty, + s"Table should have no transforms, but got: ${stTransforms.mkString(", ")}") + } + } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala index dfb766b1df778..f3b63f7914218 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala @@ -41,10 +41,12 @@ class TestPipelineDefinition(graphId: String) { // TODO: Add support for specifiedSchema // specifiedSchema: Option[StructType] = None, partitionCols: Option[Seq[String]] = None, + clusterCols: Option[Seq[String]] = None, properties: Map[String, String] = Map.empty): Unit = { val tableDetails = sc.PipelineCommand.DefineOutput.TableDetails .newBuilder() .addAllPartitionCols(partitionCols.getOrElse(Seq()).asJava) + .addAllClusteringColumns(clusterCols.getOrElse(Seq()).asJava) .putAllTableProperties(properties.asJava) .build() diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala index cb142988ce519..e5c87fa542ad1 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.connector.catalog.{ TableInfo } import org.apache.spark.sql.connector.catalog.CatalogV2Util.v2ColumnsToStructType -import org.apache.spark.sql.connector.expressions.Expressions +import org.apache.spark.sql.connector.expressions.{ClusterByTransform, Expressions} import org.apache.spark.sql.execution.command.CreateViewCommand import org.apache.spark.sql.pipelines.graph.QueryOrigin.ExceptionHelpers import org.apache.spark.sql.pipelines.util.SchemaInferenceUtils.diffSchemas @@ -266,6 +266,19 @@ object DatasetManager extends Logging { ) val mergedProperties = resolveTableProperties(table, identifier) val partitioning = table.partitionCols.toSeq.flatten.map(Expressions.identity) + val clustering = table.clusterCols.map(cols => + ClusterByTransform(cols.map(col => Expressions.column(col))) + ).toSeq + + // Validate that partition and cluster columns don't coexist + if (partitioning.nonEmpty && clustering.nonEmpty) { + throw new AnalysisException( + errorClass = "SPECIFY_CLUSTER_BY_WITH_PARTITIONED_BY_IS_NOT_ALLOWED", + messageParameters = Map.empty + ) + } + + val allTransforms = partitioning ++ clustering val existingTableOpt = if (catalog.tableExists(identifier)) { Some(catalog.loadTable(identifier)) @@ -273,15 +286,15 @@ object DatasetManager extends Logging { None } - // Error if partitioning doesn't match + // Error if partitioning/clustering doesn't match if (existingTableOpt.isDefined) { - val existingPartitioning = existingTableOpt.get.partitioning().toSeq - if (existingPartitioning != partitioning) { + val existingTransforms = existingTableOpt.get.partitioning().toSeq + if (existingTransforms != allTransforms) { throw new AnalysisException( errorClass = "CANNOT_UPDATE_PARTITION_COLUMNS", messageParameters = Map( - "existingPartitionColumns" -> existingPartitioning.mkString(", "), - "requestedPartitionColumns" -> partitioning.mkString(", ") + "existingPartitionColumns" -> existingTransforms.mkString(", "), + "requestedPartitionColumns" -> allTransforms.mkString(", ") ) ) } @@ -314,7 +327,7 @@ object DatasetManager extends Logging { new TableInfo.Builder() .withProperties(mergedProperties.asJava) .withColumns(CatalogV2Util.structTypeToV2Columns(outputSchema)) - .withPartitions(partitioning.toArray) + .withPartitions(allTransforms.toArray) .build() ) } diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala index 2c9029fdd34d6..647df79bb940c 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala @@ -264,12 +264,20 @@ class BatchTableWrite( if (destination.format.isDefined) { dataFrameWriter.format(destination.format.get) } + + // In "append" mode with saveAsTable, partition/cluster columns must be specified in query + // because the format and options of the existing table is used, and the table could + // have been created with partition columns. + if (destination.clusterCols.isDefined) { + val clusterCols = destination.clusterCols.get + dataFrameWriter.clusterBy(clusterCols.head, clusterCols.tail: _*) + } + if (destination.partitionCols.isDefined) { + dataFrameWriter.partitionBy(destination.partitionCols.get: _*) + } + dataFrameWriter .mode("append") - // In "append" mode with saveAsTable, partition columns must be specified in query - // because the format and options of the existing table is used, and the table could - // have been created with partition columns. - .partitionBy(destination.partitionCols.getOrElse(Seq.empty): _*) .saveAsTable(destination.identifier.unquotedString) } } diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala index 55a03a2d19f90..5df12be7f4cfe 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala @@ -192,6 +192,7 @@ class SqlGraphRegistrationContext( specifiedSchema = Option.when(cst.columns.nonEmpty)(StructType(cst.columns.map(_.toV1Column))), partitionCols = Option(PartitionHelper.applyPartitioning(cst.partitioning, queryOrigin)), + clusterCols = None, properties = cst.tableSpec.properties, origin = queryOrigin.copy( objectName = Option(stIdentifier.unquotedString), @@ -223,6 +224,7 @@ class SqlGraphRegistrationContext( specifiedSchema = Option.when(cst.columns.nonEmpty)(StructType(cst.columns.map(_.toV1Column))), partitionCols = Option(PartitionHelper.applyPartitioning(cst.partitioning, queryOrigin)), + clusterCols = None, properties = cst.tableSpec.properties, origin = queryOrigin.copy( objectName = Option(stIdentifier.unquotedString), @@ -273,6 +275,7 @@ class SqlGraphRegistrationContext( specifiedSchema = Option.when(cmv.columns.nonEmpty)(StructType(cmv.columns.map(_.toV1Column))), partitionCols = Option(PartitionHelper.applyPartitioning(cmv.partitioning, queryOrigin)), + clusterCols = None, properties = cmv.tableSpec.properties, origin = queryOrigin.copy( objectName = Option(mvIdentifier.unquotedString), diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala index 87e01ed2021ef..c762174e67251 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala @@ -114,6 +114,7 @@ sealed trait TableInput extends Input { * @param identifier The identifier of this table within the graph. * @param specifiedSchema The user-specified schema for this table. * @param partitionCols What columns the table should be partitioned by when materialized. + * @param clusterCols What columns the table should be clustered by when materialized. * @param normalizedPath Normalized storage location for the table based on the user-specified table * path (if not defined, we will normalize a managed storage path for it). * @param properties Table Properties to set in table metadata. @@ -124,6 +125,7 @@ case class Table( identifier: TableIdentifier, specifiedSchema: Option[StructType], partitionCols: Option[Seq[String]], + clusterCols: Option[Seq[String]], normalizedPath: Option[String], properties: Map[String, String] = Map.empty, comment: Option[String], diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala index 31afc5a27a545..ba8419eb6e9c8 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.pipelines.graph import scala.jdk.CollectionConverters._ import org.apache.spark.SparkThrowable +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} -import org.apache.spark.sql.connector.expressions.Expressions +import org.apache.spark.sql.connector.expressions.{ClusterByTransform, Expressions, FieldReference} import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.graph.DatasetManager.TableMaterializationException import org.apache.spark.sql.pipelines.utils.{BaseCoreExecutionTest, TestGraphRegistrationContext} @@ -885,4 +886,247 @@ abstract class MaterializeTablesSuite extends BaseCoreExecutionTest { storageRoot = storageRoot ) } + + test("cluster columns with user schema") { + val session = spark + import session.implicits._ + + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerTable( + "a", + query = Option(dfFlowFunc(Seq((1, 1, "x"), (2, 3, "y")).toDF("x1", "x2", "x3"))), + specifiedSchema = Option( + new StructType() + .add("x1", IntegerType) + .add("x2", IntegerType) + .add("x3", StringType) + ), + clusterCols = Option(Seq("x1", "x3")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] + val identifier = Identifier.of(Array(TestGraphRegistrationContext.DEFAULT_DATABASE), "a") + val table = catalog.loadTable(identifier) + assert( + table.columns() sameElements CatalogV2Util.structTypeToV2Columns( + new StructType() + .add("x1", IntegerType) + .add("x2", IntegerType) + .add("x3", StringType) + ) + ) + val expectedClusterTransform = ClusterByTransform( + Seq(FieldReference("x1"), FieldReference("x3")).toSeq + ) + assert(table.partitioning().contains(expectedClusterTransform)) + } + + test("specifying cluster column with existing clustered table") { + val session = spark + import session.implicits._ + + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerTable( + "t10", + query = Option(dfFlowFunc(Seq((1, true, "a"), (2, false, "b")).toDF("x", "y", "z"))), + clusterCols = Option(Seq("x", "z")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + + val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] + val identifier = Identifier.of(Array(TestGraphRegistrationContext.DEFAULT_DATABASE), "t10") + val table = catalog.loadTable(identifier) + val expectedClusterTransform = ClusterByTransform( + Seq(FieldReference("x"), FieldReference("z")).toSeq + ) + assert(table.partitioning().contains(expectedClusterTransform)) + + // Specify the same cluster columns - should work + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerFlow( + "t10", + "t10", + query = dfFlowFunc(Seq((3, true, "c"), (4, false, "d")).toDF("x", "y", "z")) + ) + registerTable("t10", clusterCols = Option(Seq("x", "z"))) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + + val table2 = catalog.loadTable(identifier) + assert(table2.partitioning().contains(expectedClusterTransform)) + + // Don't specify cluster columns when table already has them - should throw + val ex = intercept[TableMaterializationException] { + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerFlow( + "t10", + "t10", + query = dfFlowFunc(Seq((5, true, "e"), (6, false, "f")).toDF("x", "y", "z")) + ) + registerTable("t10") + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + } + assert(ex.cause.asInstanceOf[SparkThrowable].getCondition == "CANNOT_UPDATE_PARTITION_COLUMNS") + } + + test("specifying cluster column different from existing clustered table") { + val session = spark + import session.implicits._ + + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerTable( + "t11", + query = Option(dfFlowFunc(Seq((1, true, "a"), (2, false, "b")).toDF("x", "y", "z"))), + clusterCols = Option(Seq("x")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + + val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] + val identifier = Identifier.of(Array(TestGraphRegistrationContext.DEFAULT_DATABASE), "t11") + + // Specify different cluster columns - should throw + val ex = intercept[TableMaterializationException] { + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerFlow( + "t11", + "t11", + query = dfFlowFunc(Seq((3, true, "c"), (4, false, "d")).toDF("x", "y", "z")) + ) + registerTable("t11", clusterCols = Option(Seq("y"))) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + } + assert(ex.cause.asInstanceOf[SparkThrowable].getCondition == "CANNOT_UPDATE_PARTITION_COLUMNS") + + val table = catalog.loadTable(identifier) + val expectedClusterTransform = ClusterByTransform(Seq(FieldReference("x")).toSeq) + assert(table.partitioning().contains(expectedClusterTransform)) + } + + test("cluster columns only (no partitioning)") { + val session = spark + import session.implicits._ + + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerTable( + "t12", + query = Option(dfFlowFunc(Seq((1, 1, "x"), (2, 3, "y")).toDF("x1", "x2", "x3"))), + specifiedSchema = Option( + new StructType() + .add("x1", IntegerType) + .add("x2", IntegerType) + .add("x3", StringType) + ), + clusterCols = Option(Seq("x1", "x3")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] + val identifier = Identifier.of(Array(TestGraphRegistrationContext.DEFAULT_DATABASE), "t12") + val table = catalog.loadTable(identifier) + assert( + table.columns() sameElements CatalogV2Util.structTypeToV2Columns( + new StructType() + .add("x1", IntegerType) + .add("x2", IntegerType) + .add("x3", StringType) + ) + ) + + val transforms = table.partitioning() + val expectedClusterTransform = ClusterByTransform( + Seq(FieldReference("x1"), FieldReference("x3")).toSeq + ) + assert(transforms.contains(expectedClusterTransform)) + } + + test("materialized view with cluster columns") { + val session = spark + import session.implicits._ + + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerMaterializedView( + "mv1", + query = dfFlowFunc(Seq((1, 1, "x"), (2, 3, "y")).toDF("x1", "x2", "x3")), + clusterCols = Option(Seq("x1", "x2")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] + val identifier = Identifier.of(Array(TestGraphRegistrationContext.DEFAULT_DATABASE), "mv1") + val table = catalog.loadTable(identifier) + assert( + table.columns() sameElements CatalogV2Util.structTypeToV2Columns( + new StructType() + .add("x1", IntegerType) + .add("x2", IntegerType) + .add("x3", StringType) + ) + ) + val expectedClusterTransform = ClusterByTransform( + Seq(FieldReference("x1"), FieldReference("x2")).toSeq + ) + assert(table.partitioning().contains(expectedClusterTransform)) + } + + test("partition and cluster columns together should fail") { + val session = spark + import session.implicits._ + + val ex = intercept[TableMaterializationException] { + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerTable( + "invalid_table", + query = Option(dfFlowFunc(Seq((1, 1, "x"), (2, 3, "y")).toDF("x1", "x2", "x3"))), + partitionCols = Option(Seq("x2")), + clusterCols = Option(Seq("x1", "x3")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + } + assert(ex.cause.isInstanceOf[AnalysisException]) + val analysisEx = ex.cause.asInstanceOf[AnalysisException] + assert(analysisEx.errorClass.get == "SPECIFY_CLUSTER_BY_WITH_PARTITIONED_BY_IS_NOT_ALLOWED") + } + + test("cluster column that doesn't exist in table schema should fail") { + val session = spark + import session.implicits._ + + val ex = intercept[TableMaterializationException] { + materializeGraph( + new TestGraphRegistrationContext(spark) { + registerTable( + "invalid_cluster_table", + query = Option(dfFlowFunc(Seq((1, 1, "x"), (2, 3, "y")).toDF("x1", "x2", "x3"))), + clusterCols = Option(Seq("nonexistent_column")) + ) + }.resolveToDataflowGraph(), + storageRoot = storageRoot + ) + } + assert(ex.cause.isInstanceOf[AnalysisException]) + } } diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/APITest.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/APITest.scala index bb7c8e833f84b..c6b457ee04eba 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/APITest.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/APITest.scala @@ -542,6 +542,66 @@ trait APITest } } + test("Python Pipeline with cluster columns") { + val pipelineSpec = + TestPipelineSpec(include = Seq("transformations/**")) + val pipelineConfig = TestPipelineConfiguration(pipelineSpec) + val sources = Seq( + PipelineSourceFile( + name = "transformations/definition.py", + contents = """ + |from pyspark import pipelines as dp + |from pyspark.sql import DataFrame, SparkSession + |from pyspark.sql.functions import col + | + |spark = SparkSession.active() + | + |@dp.materialized_view(cluster_by = ["cluster_col1"]) + |def mv(): + | df = spark.range(10) + | df = df.withColumn("cluster_col1", col("id") % 3) + | df = df.withColumn("cluster_col2", col("id") % 2) + | return df + | + |@dp.table(cluster_by = ["cluster_col1"]) + |def st(): + | return spark.readStream.table("mv") + |""".stripMargin)) + val pipeline = createAndRunPipeline(pipelineConfig, sources) + awaitPipelineTermination(pipeline) + + // Verify tables have correct data + Seq("mv", "st").foreach { tbl => + val fullName = s"$tbl" + checkAnswer( + spark.sql(s"SELECT * FROM $fullName ORDER BY id"), + Seq( + Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 0), Row(3, 0, 1), Row(4, 1, 0), + Row(5, 2, 1), Row(6, 0, 0), Row(7, 1, 1), Row(8, 2, 0), Row(9, 0, 1) + )) + } + + // Verify clustering information is stored in catalog + val catalog = spark.sessionState.catalogManager.currentCatalog + .asInstanceOf[org.apache.spark.sql.connector.catalog.TableCatalog] + // Check materialized view has clustering transform + val mvIdentifier = org.apache.spark.sql.connector.catalog.Identifier + .of(Array("default"), "mv") + val mvTable = catalog.loadTable(mvIdentifier) + val mvTransforms = mvTable.partitioning() + assert(mvTransforms.length == 1) + assert(mvTransforms.head.name() == "cluster_by") + assert(mvTransforms.head.toString.contains("cluster_col1")) + // Check streaming table has clustering transform + val stIdentifier = org.apache.spark.sql.connector.catalog.Identifier + .of(Array("default"), "st") + val stTable = catalog.loadTable(stIdentifier) + val stTransforms = stTable.partitioning() + assert(stTransforms.length == 1) + assert(stTransforms.head.name() == "cluster_by") + assert(stTransforms.head.toString.contains("cluster_col1")) + } + /* Below tests pipeline execution configurations */ test("Pipeline with dry run") { diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala index 599aab87d1f7d..d88432d68ca3c 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala @@ -46,6 +46,7 @@ class TestGraphRegistrationContext( comment: Option[String] = None, specifiedSchema: Option[StructType] = None, partitionCols: Option[Seq[String]] = None, + clusterCols: Option[Seq[String]] = None, properties: Map[String, String] = Map.empty, baseOrigin: QueryOrigin = QueryOrigin.empty, format: Option[String] = None, @@ -58,6 +59,7 @@ class TestGraphRegistrationContext( comment, specifiedSchema, partitionCols, + clusterCols, properties, baseOrigin, format, @@ -99,6 +101,7 @@ class TestGraphRegistrationContext( comment: Option[String] = None, specifiedSchema: Option[StructType] = None, partitionCols: Option[Seq[String]] = None, + clusterCols: Option[Seq[String]] = None, properties: Map[String, String] = Map.empty, baseOrigin: QueryOrigin = QueryOrigin.empty, format: Option[String] = None, @@ -111,6 +114,7 @@ class TestGraphRegistrationContext( comment, specifiedSchema, partitionCols, + clusterCols, properties, baseOrigin, format, @@ -129,6 +133,7 @@ class TestGraphRegistrationContext( comment: Option[String], specifiedSchema: Option[StructType], partitionCols: Option[Seq[String]], + clusterCols: Option[Seq[String]], properties: Map[String, String], baseOrigin: QueryOrigin, format: Option[String], @@ -150,6 +155,7 @@ class TestGraphRegistrationContext( comment = comment, specifiedSchema = specifiedSchema, partitionCols = partitionCols, + clusterCols = clusterCols, properties = properties, origin = baseOrigin.merge( QueryOrigin( From a404945ec517a671123e523221e7b46e99a65001 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 6 Nov 2025 14:31:17 -0800 Subject: [PATCH 056/400] [SPARK-53944][K8S][CORE][FOLLOWUP] Fix SparkEnv to use `DRIVER_BIND_ADDRESS` in case of `useDriverPodIP=true` ### What changes were proposed in this pull request? This is a follow-up of the following to fix `SparkEnv` to use `DRIVER_BIND_ADDRESS` in case of `useDriverPodIP=true`. - #52650 Specifically, two places are updated. - `SparkEnv.createDriverEnv` - `RpcUtils.makeDriverRef` ### Why are the changes needed? - To prevent `Spark Driver` from advertising hostname consistently. - This PR also adds a K8s integration test by removing K8s Driver Service completely. ### Does this PR introduce _any_ user-facing change? No, this feature is not released yet. ### How was this patch tested? Pass the CIs with newly added K8s integration test. We can see that `Driver` also uses IP address like executors. Previously, `Driver` was registered with `hostname` instead of IP. Screenshot 2025-11-06 at 13 31 20 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52923 from dongjoon-hyun/SPARK-53944-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 37b19950eb6c45b8e015983c7a0ced984d8212d8) Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 11 ++++++++++- .../deploy/k8s/integrationtest/BasicTestsSuite.scala | 8 ++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 796dbf4b6d5f8..48d4faafb514e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -263,7 +263,9 @@ object SparkEnv extends Logging { s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!") assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!") val bindAddress = conf.get(DRIVER_BIND_ADDRESS) - val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS) + val useDriverPodIP = + conf.get("spark.kubernetes.executor.useDriverPodIP", "false").equalsIgnoreCase("true") + val advertiseAddress = if (useDriverPodIP) bindAddress else conf.get(DRIVER_HOST_ADDRESS) val port = conf.get(DRIVER_PORT) val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(conf)) @@ -369,6 +371,13 @@ object SparkEnv extends Logging { logInfo(log"Registering ${MDC(LogKeys.ENDPOINT_NAME, name)}") rpcEnv.setupEndpoint(name, endpointCreator) } else { + val useDriverPodIP = + conf.get("spark.kubernetes.executor.useDriverPodIP", "false").equalsIgnoreCase("true") + if (useDriverPodIP) { + logInfo(log"Use DRIVER_BIND_ADDRESS instead of DRIVER_HOST_ADDRESS because " + + log"spark.kubernetes.executor.useDriverPodIP is true") + conf.set(config.DRIVER_HOST_ADDRESS.key, conf.get(config.DRIVER_BIND_ADDRESS.key)) + } RpcUtils.makeDriverRef(name, conf, rpcEnv) } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala index d710add45eb96..22295dcaa2af7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala @@ -53,6 +53,14 @@ private[spark] trait BasicTestsSuite { k8sSuite: KubernetesSuite => runSparkPiAndVerifyCompletion() } + test("SPARK-53944: Run SparkPi without driver service", k8sTestTag) { + sparkAppConf.set( + "spark.kubernetes.driver.pod.excludedFeatureSteps", + "org.apache.spark.deploy.k8s.features.DriverServiceFeatureStep") + sparkAppConf.set("spark.kubernetes.executor.useDriverPodIP", "true") + runSparkPiAndVerifyCompletion() + } + test("Run SparkPi with no resources & statefulset allocation", k8sTestTag) { sparkAppConf.set("spark.kubernetes.allocation.pods.allocator", "statefulset") runSparkPiAndVerifyCompletion() From 95ea4ce11842718c99be4754e05312b6c0586df9 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Fri, 7 Nov 2025 08:11:20 +0900 Subject: [PATCH 057/400] [MINOR][DOCS][CONNECT] Replace pyspark with pyspark-client in the Connect overview ### What changes were proposed in this pull request? Spark Connect Overview doc should reference the non-JVM pyspark-client pip package instead of `pyspark[connect]` which includes a complete SPARK_HOME with JVM jars ### Why are the changes needed? Spark Connect Overview is a top search result and should encourage using a package with the minimum footprint. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Previewed the doc on Github ### Was this patch authored or co-authored using generative AI tooling? No Closes #52901 from gerashegalov/patch-1. Authored-by: Gera Shegalov Signed-off-by: Hyukjin Kwon (cherry picked from commit 2f04e783b7e30c2071c80f1f3ed5ff9e319505de) Signed-off-by: Hyukjin Kwon --- docs/spark-connect-overview.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/spark-connect-overview.md b/docs/spark-connect-overview.md index f01ebf1b54f76..3c15153e03053 100644 --- a/docs/spark-connect-overview.md +++ b/docs/spark-connect-overview.md @@ -284,11 +284,11 @@ The connection may also be programmatically created using _SparkSession#builder_
-First, install PySpark with `pip install pyspark[connect]=={{site.SPARK_VERSION_SHORT}}` or if building a packaged PySpark application/library, +First, install PySpark with `pip install pyspark-client=={{site.SPARK_VERSION_SHORT}}` or if building a packaged PySpark application/library, add it your setup.py file as: {% highlight python %} install_requires=[ -'pyspark[connect]=={{site.SPARK_VERSION_SHORT}}' +'pyspark-client=={{site.SPARK_VERSION_SHORT}}' ] {% endhighlight %} From 5ed23dc6614fb10ea9a5d3e46c2ddc04fcda41d2 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Thu, 6 Nov 2025 20:21:43 -0800 Subject: [PATCH 058/400] [SPARK-54201][GEO][SQL] Allow casting from GeographyType(srid) to GeographyType(ANY) ### What changes were proposed in this pull request? This PR allows casting fixed SRID type `GEOGRAPHY()` to mixed SRID type `GEOGRAPHY(ANY)`. ### Why are the changes needed? Enable explicit casting between geography types. ### Does this PR introduce _any_ user-facing change? Yes, casting `GEOGRAPHY()` to `GEOGRAPHY(ANY)` is now allowed. ### How was this patch tested? Added new unit tests: - `StUtilsSuite` - `CastSuiteBase` Added new e2e SQL tests: - `st-functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52906 from uros-db/geo-cast-geography_any. Authored-by: Uros Bojanic Signed-off-by: Dongjoon Hyun (cherry picked from commit 941c24bbebe2ff956cf81d6a7385b220fdabca25) Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/Cast.scala | 8 ++++ .../catalyst/expressions/CastSuiteBase.scala | 21 +++++++++ .../nonansi/st-functions.sql.out | 29 +++++++++++++ .../analyzer-results/st-functions.sql.out | 29 +++++++++++++ .../sql-tests/inputs/st-functions.sql | 5 +++ .../results/nonansi/st-functions.sql.out | 32 ++++++++++++++ .../sql-tests/results/st-functions.sql.out | 32 ++++++++++++++ .../apache/spark/sql/STExpressionsSuite.scala | 43 +++++++++++++++++++ 8 files changed, 199 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 974cdfe1b012f..22f1ad73a0e84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -164,6 +164,9 @@ object Cast extends QueryErrorsBase { case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true + // Casts from concrete GEOGRAPHY(srid) to mixed GEOGRAPHY(ANY) is allowed. + case (gt1: GeographyType, gt2: GeographyType) if !gt1.isMixedSrid && gt2.isMixedSrid => + true // Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed. case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid => true @@ -294,6 +297,9 @@ object Cast extends QueryErrorsBase { case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true + // Casts from concrete GEOGRAPHY(srid) to mixed GEOGRAPHY(ANY) is allowed. + case (gt1: GeographyType, gt2: GeographyType) if !gt1.isMixedSrid && gt2.isMixedSrid => + true // Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed. case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid => true @@ -1232,6 +1238,7 @@ case class Cast( case FloatType => castToFloat(from) case LongType => castToLong(from) case DoubleType => castToDouble(from) + case _: GeographyType => identity case _: GeometryType => castToGeometry(from) case array: ArrayType => castArray(from.asInstanceOf[ArrayType].elementType, array.elementType) @@ -1341,6 +1348,7 @@ case class Cast( case FloatType => castToFloatCode(from, ctx) case LongType => castToLongCode(from, ctx) case DoubleType => castToDoubleCode(from, ctx) + case _: GeographyType => (c, evPrim, _) => code"$evPrim = $c;" case _: GeometryType => castToGeometryCode(from) case array: ArrayType => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index bf28e2e7eeb79..2220f56255f0a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -1491,6 +1491,27 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { // The following tests are confirming the behavior of casting between geospatial types. + test("Casting GeographyType to GeographyType") { + // Casting from fixed SRID GEOGRAPHY() to mixed SRID GEOGRAPHY(ANY) is always allowed. + // Type casting is always safe in this direction, so no additional constraints are imposed. + // Casting from mixed SRID GEOGRAPHY(ANY) to fixed SRID GEOGRAPHY() is not allowed. + // Type casting can be unsafe in this direction, because per-row SRID values may be different. + + // Valid cast test cases. + val canCastTestCases: Seq[(DataType, DataType)] = Seq( + (GeographyType(4326), GeographyType("ANY")) + ) + // Iterate over the test cases and verify casting. + canCastTestCases.foreach { case (fromType, toType) => + // Cast can be performed from `fromType` to `toType`. + assert(Cast.canCast(fromType, toType)) + assert(Cast.canAnsiCast(fromType, toType)) + // Cast cannot be performed from `toType` to `fromType`. + assert(!Cast.canCast(toType, fromType)) + assert(!Cast.canAnsiCast(toType, fromType)) + } + } + test("Casting GeographyType to GeometryType") { // Casting from GEOGRAPHY to GEOMETRY is only allowed if the SRIDs are the same. diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out index 55bb5f6ab7b00..1d2094f3b9ef1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out @@ -66,6 +66,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result +-- !query analysis +Project [hex(st_asbinary(cast(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040) as geography(any)))) AS result#x] ++- OneRowRelation + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"", + "srcType" : "\"GEOGRAPHY(ANY)\"", + "targetType" : "\"GEOGRAPHY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 109, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out index 55bb5f6ab7b00..1d2094f3b9ef1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out @@ -66,6 +66,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result +-- !query analysis +Project [hex(st_asbinary(cast(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040) as geography(any)))) AS result#x] ++- OneRowRelation + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"", + "srcType" : "\"GEOGRAPHY(ANY)\"", + "targetType" : "\"GEOGRAPHY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 109, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql index b824be2211228..7f4b77c7e0f9c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql @@ -13,6 +13,11 @@ INSERT INTO geodata VALUES SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS STRING) AS result; SELECT CAST(X'0101000000000000000000f03f0000000000000040' AS GEOMETRY(4326)) AS result; +-- Casting GEOGRAPHY() to GEOGRAPHY(ANY) is allowed. +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result; +-- Casting GEOGRAPHY(ANY) to GEOGRAPHY() is not allowed. +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result; + -- Casting GEOGRAPHY to GEOMETRY is allowed only if SRIDs match. SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result; -- Error handling: mismatched SRIDs. diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out index c5b6daea764bb..088c49cb030a3 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out @@ -73,6 +73,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result +-- !query schema +struct +-- !query output +0101000000000000000000F03F0000000000000040 + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"", + "srcType" : "\"GEOGRAPHY(ANY)\"", + "targetType" : "\"GEOGRAPHY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 109, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out index c5b6daea764bb..088c49cb030a3 100644 --- a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out @@ -73,6 +73,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result +-- !query schema +struct +-- !query output +0101000000000000000000F03F0000000000000040 + + +-- !query +SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"", + "srcType" : "\"GEOGRAPHY(ANY)\"", + "targetType" : "\"GEOGRAPHY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 109, + "fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 700f9700ebc90..4c34a5c8f78f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -31,6 +31,7 @@ class STExpressionsSuite // Private common constants used across several tests. private final val defaultGeographySrid: Int = ExpressionDefaults.DEFAULT_GEOGRAPHY_SRID private final val defaultGeographyType: DataType = GeographyType(defaultGeographySrid) + private final val mixedSridGeographyType: DataType = GeographyType("ANY") private final val defaultGeometrySrid: Int = ExpressionDefaults.DEFAULT_GEOMETRY_SRID private final val defaultGeometryType: DataType = GeometryType(defaultGeometrySrid) @@ -39,6 +40,48 @@ class STExpressionsSuite assert(sql(query).schema.fields.head.dataType.sameType(expectedDataType)) } + /** Geospatial type casting. */ + + test("Cast GEOGRAPHY(srid) to GEOGRAPHY(ANY)") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + val wkb = Hex.unhex(wkbString.getBytes()) + val wkbLiteral = Literal.create(wkb, BinaryType) + + // Construct the input GEOGRAPHY expression. + val geogExpr = ST_GeogFromWKB(wkbLiteral) + assert(geogExpr.dataType.sameType(defaultGeographyType)) + checkEvaluation(ST_AsBinary(geogExpr), wkb) + // Cast the GEOGRAPHY with fixed SRID to GEOGRAPHY with mixed SRID. + val castExpr = Cast(geogExpr, mixedSridGeographyType) + assert(castExpr.dataType.sameType(mixedSridGeographyType)) + checkEvaluation(ST_AsBinary(castExpr), wkb) + + // Construct the input GEOGRAPHY SQL query, using WKB literal. + val geogQueryLit: String = s"ST_GeogFromWKB(X'$wkbString')" + assertType(s"SELECT $geogQueryLit", defaultGeographyType) + checkAnswer(sql(s"SELECT ST_AsBinary($geogQueryLit)"), Row(wkb)) + // Cast the GEOGRAPHY with fixed SRID to GEOGRAPHY with mixed SRID. + val castQueryLit = s"$geogQueryLit::GEOGRAPHY(ANY)" + assertType(s"SELECT $castQueryLit", mixedSridGeographyType) + checkAnswer(sql(s"SELECT ST_AsBinary($castQueryLit)"), Row(wkb)) + + withTable("tbl") { + // Construct the test table with WKB. + sql(s"CREATE TABLE tbl (wkb BINARY)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString')") + + // Construct the input GEOGRAPHY SQL query, using WKB column. + val geogQueryCol: String = s"ST_GeogFromWKB(wkb)" + assertType(s"SELECT $geogQueryCol FROM tbl", defaultGeographyType) + checkAnswer(sql(s"SELECT ST_AsBinary($geogQueryCol) FROM tbl"), Row(wkb)) + // Cast the GEOGRAPHY with fixed SRID to GEOGRAPHY with mixed SRID. + val castQueryCol = s"$geogQueryCol::GEOGRAPHY(ANY)" + assertType(s"SELECT $castQueryCol FROM tbl", mixedSridGeographyType) + checkAnswer(sql(s"SELECT ST_AsBinary($castQueryCol) FROM tbl"), Row(wkb)) + } + } + /** ST reader/writer expressions. */ test("ST_AsBinary") { From d4aaeaea49f4dba0d57c66eb85ae92c1dbd6ba1a Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 7 Nov 2025 16:56:02 +0800 Subject: [PATCH 059/400] [SPARK-54190][BUILD] Guava dependency governance ### What changes were proposed in this pull request? Remove `connect.guava.version` and use the unified `guava.version`. Strip the unused transitive dependencies of Guava: as mentioned in https://github.com/google/guava/wiki/UseGuavaInYourBuild > Guava has one dependency that is needed for linkage at runtime: > com.google.guava:failureaccess: Remove shaded Guava classes from `spark-connect` jar (reuse shaded Guava included in `spark-network-common`) Fix the shading leaks of the `spark-connect-jvm-client` jar ### Why are the changes needed? 1. Simplify Guava dependency management - now Spark uses a unified Guava version everywhere. 2. Reduce package size, spark-connect jar becomes smaller before (master branch) ``` $ ll jars/spark-connect_2.13-4.2.0-SNAPSHOT.jar -rw-r--r-- 1 chengpan staff 17M Nov 5 11:23 jars/spark-connect_2.13-4.2.0-SNAPSHOT.jar ``` after (this PR) ``` $ ll jars/spark-connect_2.13-4.2.0-SNAPSHOT.jar -rw-r--r-- 1 chengpan staff 13M Nov 5 12:01 jars/spark-connect_2.13-4.2.0-SNAPSHOT.jar ``` 2. Fix the shading leaks for `spark-connect-jvm-client` jar before (master branch) ``` $ jar tf jars/connect-repl/spark-connect-client-jvm_2.13-4.2.0-SNAPSHOT.jar | grep '.class$' | grep -v 'org/apache/spark' | grep -v 'org/sparkproject' | grep -v 'META-INF' javax/annotation/CheckForNull.class javax/annotation/CheckForSigned.class ... ``` after (this PR) ``` $ jar tf jars/connect-repl/spark-connect-client-jvm_2.13-4.2.0-SNAPSHOT.jar | grep '.class$' | grep -v 'org/apache/spark' | grep -v 'org/sparkproject' | grep -v 'META-INF' ``` ### Does this PR introduce _any_ user-facing change? Reduce potential class conflict issues for users who use `spark-connect-jvm-client`. ### How was this patch tested? Manually checked, see the above section. Also, manually tested the Connect Server, and Connect JVM client via BeeLine. ``` $ dev/make-distribution.sh --tgz --name guava -Pyarn -Pkubernetes -Phadoop-3 -Phive -Phive-thriftserver $ cd dist $ SPARK_NO_DAEMONIZE=1 sbin/start-connect-server.sh ``` ``` $ SPARK_CONNECT_BEELINE=1 bin/beeline -u jdbc:sc://localhost:15002 -e "select 'Hello, Spark Connect!', version() as server_version;" WARNING: Using incubator modules: jdk.incubator.vector Connecting to jdbc:sc://localhost:15002 Connected to: Apache Spark Connect Server (version 4.2.0-SNAPSHOT) Driver: Apache Spark Connect JDBC Driver (version 4.2.0-SNAPSHOT) Error: Requested transaction isolation level REPEATABLE_READ is not supported (state=,code=0) Using Spark's default log4j profile: org/apache/spark/log4j2-defaults.properties 25/11/05 13:30:03 WARN Utils: Your hostname, H27212-MAC-01.local, resolves to a loopback address: 127.0.0.1; using 10.242.159.140 instead (on interface en0) 25/11/05 13:30:03 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +------------------------+-------------------------------------------------+ | Hello, Spark Connect! | server_version | +------------------------+-------------------------------------------------+ | Hello, Spark Connect! | 4.2.0 0ea7f5599c5dcc169b0724caa48d5530c39dbefb | +------------------------+-------------------------------------------------+ 1 row selected (0.09 seconds) Beeline version 2.3.10 by Apache Hive Closing: 0: jdbc:sc://localhost:15002 ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52873 from pan3793/guava-govern. Authored-by: Cheng Pan Signed-off-by: yangjie01 (cherry picked from commit a8d128c7acaa1fdf483f3eb13a1cf22f28f0b3f7) Signed-off-by: yangjie01 --- LICENSE-binary | 3 - assembly/pom.xml | 14 + common/network-common/pom.xml | 5 + dev/deps/spark-deps-hadoop-3-hive-2.3 | 4 - licenses-binary/LICENSE-check-qual.txt | 413 ------------------------- pom.xml | 9 +- project/SparkBuild.scala | 56 ++-- sql/connect/client/jdbc/pom.xml | 6 - sql/connect/client/jvm/pom.xml | 10 - sql/connect/server/pom.xml | 33 +- 10 files changed, 59 insertions(+), 494 deletions(-) delete mode 100644 licenses-binary/LICENSE-check-qual.txt diff --git a/LICENSE-binary b/LICENSE-binary index 95087a0a0de23..fcc54c51bc820 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -215,10 +215,8 @@ com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter com.google.code.findbugs:jsr305 com.google.code.gson:gson com.google.crypto.tink:tink -com.google.errorprone:error_prone_annotations com.google.flatbuffers:flatbuffers-java com.google.guava:guava -com.google.j2objc:j2objc-annotations com.jamesmurty.utils:java-xmlbuilder com.ning:compress-lzf com.squareup.okhttp3:logging-interceptor @@ -478,7 +476,6 @@ dev.ludovic.netlib:blas dev.ludovic.netlib:arpack dev.ludovic.netlib:lapack net.razorvine:pickle -org.checkerframework:checker-qual org.typelevel:algebra_2.13:jar org.typelevel:cats-kernel_2.13 org.typelevel:spire_2.13 diff --git a/assembly/pom.xml b/assembly/pom.xml index 0e6012062313e..8de2333671abf 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -142,11 +142,25 @@ Because we don't shade dependencies anymore, we need to restore Guava to compile scope so that the libraries Spark depend on have it available. We'll package the version that Spark uses which is not the same as Hadoop dependencies, but works. + As mentioned in https://github.com/google/guava/wiki/UseGuavaInYourBuild + Guava has one dependency that is needed for linkage at runtime: + com.google.guava:failureaccess: --> com.google.guava guava ${hadoop.deps.scope} + + + * + * + + + + + com.google.guava + failureaccess + ${hadoop.deps.scope} diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 51b782920e6d9..f8283b5ddb07c 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -123,6 +123,11 @@ guava compile + + com.google.guava + failureaccess + compile + org.apache.commons commons-crypto diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 73f36fb161c66..3bf4d78f5023b 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -33,7 +33,6 @@ breeze-macros_2.13/2.1.0//breeze-macros_2.13-2.1.0.jar breeze_2.13/2.1.0//breeze_2.13-2.1.0.jar bundle/2.29.52//bundle-2.29.52.jar cats-kernel_2.13/2.8.0//cats-kernel_2.13-2.8.0.jar -checker-qual/3.43.0//checker-qual-3.43.0.jar chill-java/0.10.0//chill-java-0.10.0.jar chill_2.13/0.10.0//chill_2.13-0.10.0.jar commons-cli/1.10.0//commons-cli-1.10.0.jar @@ -62,7 +61,6 @@ derby/10.16.1.1//derby-10.16.1.1.jar derbyshared/10.16.1.1//derbyshared-10.16.1.1.jar derbytools/10.16.1.1//derbytools-10.16.1.1.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar -error_prone_annotations/2.36.0//error_prone_annotations-2.36.0.jar esdk-obs-java/3.20.4.2//esdk-obs-java-3.20.4.2.jar failureaccess/1.0.2//failureaccess-1.0.2.jar flatbuffers-java/25.2.10//flatbuffers-java-25.2.10.jar @@ -102,7 +100,6 @@ icu4j/77.1//icu4j-77.1.jar ini4j/0.5.4//ini4j-0.5.4.jar istack-commons-runtime/4.1.2//istack-commons-runtime-4.1.2.jar ivy/2.5.3//ivy-2.5.3.jar -j2objc-annotations/3.0.0//j2objc-annotations-3.0.0.jar jackson-annotations/2.20//jackson-annotations-2.20.jar jackson-core/2.20.0//jackson-core-2.20.0.jar jackson-databind/2.20.0//jackson-databind-2.20.0.jar @@ -185,7 +182,6 @@ lapack/3.0.4//lapack-3.0.4.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.16.0//libthrift-0.16.0.jar -listenablefuture/9999.0-empty-to-avoid-conflict-with-guava//listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar log4j-1.2-api/2.24.3//log4j-1.2-api-2.24.3.jar log4j-api/2.24.3//log4j-api-2.24.3.jar log4j-core/2.24.3//log4j-core-2.24.3.jar diff --git a/licenses-binary/LICENSE-check-qual.txt b/licenses-binary/LICENSE-check-qual.txt deleted file mode 100644 index d542ab3ec3ed8..0000000000000 --- a/licenses-binary/LICENSE-check-qual.txt +++ /dev/null @@ -1,413 +0,0 @@ -The Checker Framework -Copyright 2004-present by the Checker Framework developers - - -Most of the Checker Framework is licensed under the GNU General Public -License, version 2 (GPL2), with the classpath exception. The text of this -license appears below. This is the same license used for OpenJDK. - -A few parts of the Checker Framework have more permissive licenses, notably -the parts that you might want to include with your own program. - - * The annotations and utility files are licensed under the MIT License. - (The text of this license also appears below.) This applies to - checker-qual*.jar and checker-util.jar and all the files that appear in - them, which is all files in checker-qual and checker-util directories. - It also applies to the cleanroom implementations of - third-party annotations (in checker/src/testannotations/, - framework/src/main/java/org/jmlspecs/, and - framework/src/main/java/com/google/). - -The Checker Framework includes annotations for some libraries. Those in -.astub files use the MIT License. Those in https://github.com/typetools/jdk -(which appears in the annotated-jdk directory of file checker.jar) use the -GPL2 license. - -Some external libraries that are included with the Checker Framework -distribution have different licenses. Here are some examples. - - * JavaParser is dual licensed under the LGPL or the Apache license -- you - may use it under whichever one you want. (The JavaParser source code - contains a file with the text of the GPL, but it is not clear why, since - JavaParser does not use the GPL.) See - https://github.com/typetools/stubparser . - - * Annotation Tools (https://github.com/typetools/annotation-tools) uses - the MIT license. - - * Libraries in plume-lib (https://github.com/plume-lib/) are licensed - under the MIT License. - -=========================================================================== - -The GNU General Public License (GPL) - -Version 2, June 1991 - -Copyright (C) 1989, 1991 Free Software Foundation, Inc. -59 Temple Place, Suite 330, Boston, MA 02111-1307 USA - -Everyone is permitted to copy and distribute verbatim copies of this license -document, but changing it is not allowed. - -Preamble - -The licenses for most software are designed to take away your freedom to share -and change it. By contrast, the GNU General Public License is intended to -guarantee your freedom to share and change free software--to make sure the -software is free for all its users. This General Public License applies to -most of the Free Software Foundation's software and to any other program whose -authors commit to using it. (Some other Free Software Foundation software is -covered by the GNU Library General Public License instead.) You can apply it to -your programs, too. - -When we speak of free software, we are referring to freedom, not price. Our -General Public Licenses are designed to make sure that you have the freedom to -distribute copies of free software (and charge for this service if you wish), -that you receive source code or can get it if you want it, that you can change -the software or use pieces of it in new free programs; and that you know you -can do these things. - -To protect your rights, we need to make restrictions that forbid anyone to deny -you these rights or to ask you to surrender the rights. These restrictions -translate to certain responsibilities for you if you distribute copies of the -software, or if you modify it. - -For example, if you distribute copies of such a program, whether gratis or for -a fee, you must give the recipients all the rights that you have. You must -make sure that they, too, receive or can get the source code. And you must -show them these terms so they know their rights. - -We protect your rights with two steps: (1) copyright the software, and (2) -offer you this license which gives you legal permission to copy, distribute -and/or modify the software. - -Also, for each author's protection and ours, we want to make certain that -everyone understands that there is no warranty for this free software. If the -software is modified by someone else and passed on, we want its recipients to -know that what they have is not the original, so that any problems introduced -by others will not reflect on the original authors' reputations. - -Finally, any free program is threatened constantly by software patents. We -wish to avoid the danger that redistributors of a free program will -individually obtain patent licenses, in effect making the program proprietary. -To prevent this, we have made it clear that any patent must be licensed for -everyone's free use or not licensed at all. - -The precise terms and conditions for copying, distribution and modification -follow. - -TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - -0. This License applies to any program or other work which contains a notice -placed by the copyright holder saying it may be distributed under the terms of -this General Public License. The "Program", below, refers to any such program -or work, and a "work based on the Program" means either the Program or any -derivative work under copyright law: that is to say, a work containing the -Program or a portion of it, either verbatim or with modifications and/or -translated into another language. (Hereinafter, translation is included -without limitation in the term "modification".) Each licensee is addressed as -"you". - -Activities other than copying, distribution and modification are not covered by -this License; they are outside its scope. The act of running the Program is -not restricted, and the output from the Program is covered only if its contents -constitute a work based on the Program (independent of having been made by -running the Program). Whether that is true depends on what the Program does. - -1. You may copy and distribute verbatim copies of the Program's source code as -you receive it, in any medium, provided that you conspicuously and -appropriately publish on each copy an appropriate copyright notice and -disclaimer of warranty; keep intact all the notices that refer to this License -and to the absence of any warranty; and give any other recipients of the -Program a copy of this License along with the Program. - -You may charge a fee for the physical act of transferring a copy, and you may -at your option offer warranty protection in exchange for a fee. - -2. You may modify your copy or copies of the Program or any portion of it, thus -forming a work based on the Program, and copy and distribute such modifications -or work under the terms of Section 1 above, provided that you also meet all of -these conditions: - - a) You must cause the modified files to carry prominent notices stating - that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in whole or - in part contains or is derived from the Program or any part thereof, to be - licensed as a whole at no charge to all third parties under the terms of - this License. - - c) If the modified program normally reads commands interactively when run, - you must cause it, when started running for such interactive use in the - most ordinary way, to print or display an announcement including an - appropriate copyright notice and a notice that there is no warranty (or - else, saying that you provide a warranty) and that users may redistribute - the program under these conditions, and telling the user how to view a copy - of this License. (Exception: if the Program itself is interactive but does - not normally print such an announcement, your work based on the Program is - not required to print an announcement.) - -These requirements apply to the modified work as a whole. If identifiable -sections of that work are not derived from the Program, and can be reasonably -considered independent and separate works in themselves, then this License, and -its terms, do not apply to those sections when you distribute them as separate -works. But when you distribute the same sections as part of a whole which is a -work based on the Program, the distribution of the whole must be on the terms -of this License, whose permissions for other licensees extend to the entire -whole, and thus to each and every part regardless of who wrote it. - -Thus, it is not the intent of this section to claim rights or contest your -rights to work written entirely by you; rather, the intent is to exercise the -right to control the distribution of derivative or collective works based on -the Program. - -In addition, mere aggregation of another work not based on the Program with the -Program (or with a work based on the Program) on a volume of a storage or -distribution medium does not bring the other work under the scope of this -License. - -3. You may copy and distribute the Program (or a work based on it, under -Section 2) in object code or executable form under the terms of Sections 1 and -2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable source - code, which must be distributed under the terms of Sections 1 and 2 above - on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three years, to - give any third party, for a charge no more than your cost of physically - performing source distribution, a complete machine-readable copy of the - corresponding source code, to be distributed under the terms of Sections 1 - and 2 above on a medium customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer to - distribute corresponding source code. (This alternative is allowed only - for noncommercial distribution and only if you received the program in - object code or executable form with such an offer, in accord with - Subsection b above.) - -The source code for a work means the preferred form of the work for making -modifications to it. For an executable work, complete source code means all -the source code for all modules it contains, plus any associated interface -definition files, plus the scripts used to control compilation and installation -of the executable. However, as a special exception, the source code -distributed need not include anything that is normally distributed (in either -source or binary form) with the major components (compiler, kernel, and so on) -of the operating system on which the executable runs, unless that component -itself accompanies the executable. - -If distribution of executable or object code is made by offering access to copy -from a designated place, then offering equivalent access to copy the source -code from the same place counts as distribution of the source code, even though -third parties are not compelled to copy the source along with the object code. - -4. You may not copy, modify, sublicense, or distribute the Program except as -expressly provided under this License. Any attempt otherwise to copy, modify, -sublicense or distribute the Program is void, and will automatically terminate -your rights under this License. However, parties who have received copies, or -rights, from you under this License will not have their licenses terminated so -long as such parties remain in full compliance. - -5. You are not required to accept this License, since you have not signed it. -However, nothing else grants you permission to modify or distribute the Program -or its derivative works. These actions are prohibited by law if you do not -accept this License. Therefore, by modifying or distributing the Program (or -any work based on the Program), you indicate your acceptance of this License to -do so, and all its terms and conditions for copying, distributing or modifying -the Program or works based on it. - -6. Each time you redistribute the Program (or any work based on the Program), -the recipient automatically receives a license from the original licensor to -copy, distribute or modify the Program subject to these terms and conditions. -You may not impose any further restrictions on the recipients' exercise of the -rights granted herein. You are not responsible for enforcing compliance by -third parties to this License. - -7. If, as a consequence of a court judgment or allegation of patent -infringement or for any other reason (not limited to patent issues), conditions -are imposed on you (whether by court order, agreement or otherwise) that -contradict the conditions of this License, they do not excuse you from the -conditions of this License. If you cannot distribute so as to satisfy -simultaneously your obligations under this License and any other pertinent -obligations, then as a consequence you may not distribute the Program at all. -For example, if a patent license would not permit royalty-free redistribution -of the Program by all those who receive copies directly or indirectly through -you, then the only way you could satisfy both it and this License would be to -refrain entirely from distribution of the Program. - -If any portion of this section is held invalid or unenforceable under any -particular circumstance, the balance of the section is intended to apply and -the section as a whole is intended to apply in other circumstances. - -It is not the purpose of this section to induce you to infringe any patents or -other property right claims or to contest validity of any such claims; this -section has the sole purpose of protecting the integrity of the free software -distribution system, which is implemented by public license practices. Many -people have made generous contributions to the wide range of software -distributed through that system in reliance on consistent application of that -system; it is up to the author/donor to decide if he or she is willing to -distribute software through any other system and a licensee cannot impose that -choice. - -This section is intended to make thoroughly clear what is believed to be a -consequence of the rest of this License. - -8. If the distribution and/or use of the Program is restricted in certain -countries either by patents or by copyrighted interfaces, the original -copyright holder who places the Program under this License may add an explicit -geographical distribution limitation excluding those countries, so that -distribution is permitted only in or among countries not thus excluded. In -such case, this License incorporates the limitation as if written in the body -of this License. - -9. The Free Software Foundation may publish revised and/or new versions of the -General Public License from time to time. Such new versions will be similar in -spirit to the present version, but may differ in detail to address new problems -or concerns. - -Each version is given a distinguishing version number. If the Program -specifies a version number of this License which applies to it and "any later -version", you have the option of following the terms and conditions either of -that version or of any later version published by the Free Software Foundation. -If the Program does not specify a version number of this License, you may -choose any version ever published by the Free Software Foundation. - -10. If you wish to incorporate parts of the Program into other free programs -whose distribution conditions are different, write to the author to ask for -permission. For software which is copyrighted by the Free Software Foundation, -write to the Free Software Foundation; we sometimes make exceptions for this. -Our decision will be guided by the two goals of preserving the free status of -all derivatives of our free software and of promoting the sharing and reuse of -software generally. - -NO WARRANTY - -11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR -THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE -STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE -PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, -INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND -FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND -PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, -YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. - -12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL -ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE -PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY -GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR -INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA -BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A -FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER -OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. - -END OF TERMS AND CONDITIONS - -How to Apply These Terms to Your New Programs - -If you develop a new program, and you want it to be of the greatest possible -use to the public, the best way to achieve this is to make it free software -which everyone can redistribute and change under these terms. - -To do so, attach the following notices to the program. It is safest to attach -them to the start of each source file to most effectively convey the exclusion -of warranty; and each file should have at least the "copyright" line and a -pointer to where the full notice is found. - - One line to give the program's name and a brief idea of what it does. - - Copyright (C) - - This program is free software; you can redistribute it and/or modify it - under the terms of the GNU General Public License as published by the Free - Software Foundation; either version 2 of the License, or (at your option) - any later version. - - This program is distributed in the hope that it will be useful, but WITHOUT - ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for - more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., 59 - Temple Place, Suite 330, Boston, MA 02111-1307 USA - -Also add information on how to contact you by electronic and paper mail. - -If the program is interactive, make it output a short notice like this when it -starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author Gnomovision comes - with ABSOLUTELY NO WARRANTY; for details type 'show w'. This is free - software, and you are welcome to redistribute it under certain conditions; - type 'show c' for details. - -The hypothetical commands 'show w' and 'show c' should show the appropriate -parts of the General Public License. Of course, the commands you use may be -called something other than 'show w' and 'show c'; they could even be -mouse-clicks or menu items--whatever suits your program. - -You should also get your employer (if you work as a programmer) or your school, -if any, to sign a "copyright disclaimer" for the program, if necessary. Here -is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the program - 'Gnomovision' (which makes passes at compilers) written by James Hacker. - - signature of Ty Coon, 1 April 1989 - - Ty Coon, President of Vice - -This General Public License does not permit incorporating your program into -proprietary programs. If your program is a subroutine library, you may -consider it more useful to permit linking proprietary applications with the -library. If this is what you want to do, use the GNU Library General Public -License instead of this License. - - -"CLASSPATH" EXCEPTION TO THE GPL - -Certain source files distributed by Oracle America and/or its affiliates are -subject to the following clarification and special exception to the GPL, but -only where Oracle has expressly included in the particular source file's header -the words "Oracle designates this particular file as subject to the "Classpath" -exception as provided by Oracle in the LICENSE file that accompanied this code." - - Linking this library statically or dynamically with other modules is making - a combined work based on this library. Thus, the terms and conditions of - the GNU General Public License cover the whole combination. - - As a special exception, the copyright holders of this library give you - permission to link this library with independent modules to produce an - executable, regardless of the license terms of these independent modules, - and to copy and distribute the resulting executable under terms of your - choice, provided that you also meet, for each linked independent module, - the terms and conditions of the license of that module. An independent - module is a module which is not derived from or based on this library. If - you modify this library, you may extend this exception to your version of - the library, but you are not obligated to do so. If you do not wish to do - so, delete this exception statement from your version. - -=========================================================================== - -MIT License: - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in -all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -THE SOFTWARE. - -=========================================================================== \ No newline at end of file diff --git a/pom.xml b/pom.xml index b428823b3cb51..d2469b11b3ab2 100644 --- a/pom.xml +++ b/pom.xml @@ -199,6 +199,7 @@ 2.12.1 4.1.17 33.4.0-jre + 1.0.2 2.11.0 3.1.9 3.0.18 @@ -303,8 +304,6 @@ true - 33.4.0-jre - 1.0.2 1.76.0 1.1.4 @@ -610,6 +609,12 @@ ${guava.version} provided + + com.google.guava + failureaccess + ${guava.failureaccess.version} + provided + org.jpmml pmml-model diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 253893cc225b5..90c6b6c844a9f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -672,7 +672,7 @@ object SparkConnectCommon { libraryDependencies ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] val guavaFailureaccessVersion = SbtPomKeys.effectivePom.value.getProperties.get( "guava.failureaccess.version").asInstanceOf[String] @@ -690,7 +690,7 @@ object SparkConnectCommon { dependencyOverrides ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] val guavaFailureaccessVersion = SbtPomKeys.effectivePom.value.getProperties.get( "guava.failureaccess.version").asInstanceOf[String] @@ -758,7 +758,7 @@ object SparkConnect { libraryDependencies ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] val guavaFailureaccessVersion = SbtPomKeys.effectivePom.value.getProperties.get( "guava.failureaccess.version").asInstanceOf[String] @@ -772,7 +772,7 @@ object SparkConnect { dependencyOverrides ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] val guavaFailureaccessVersion = SbtPomKeys.effectivePom.value.getProperties.get( "guava.failureaccess.version").asInstanceOf[String] @@ -790,18 +790,15 @@ object SparkConnect { // Exclude `scala-library` from assembly. (assembly / assemblyPackageScala / assembleArtifact) := false, - // SPARK-46733: Include `spark-connect-*.jar`, `unused-*.jar`,`guava-*.jar`, - // `failureaccess-*.jar`, `annotations-*.jar`, `grpc-*.jar`, `protobuf-*.jar`, - // `gson-*.jar`, `error_prone_annotations-*.jar`, `j2objc-annotations-*.jar`, - // `animal-sniffer-annotations-*.jar`, `perfmark-api-*.jar`, - // `proto-google-common-protos-*.jar` in assembly. + // SPARK-46733: Include `spark-connect-*.jar`, `unused-*.jar`, `annotations-*.jar`, + // `grpc-*.jar`, `protobuf-*.jar`, `gson-*.jar`, `animal-sniffer-annotations-*.jar`, + // `perfmark-api-*.jar`, `proto-google-common-protos-*.jar` in assembly. // This needs to be consistent with the content of `maven-shade-plugin`. (assembly / assemblyExcludedJars) := { val cp = (assembly / fullClasspath).value - val validPrefixes = Set("spark-connect", "unused-", "guava-", "failureaccess-", - "annotations-", "grpc-", "protobuf-", "gson", "error_prone_annotations", - "j2objc-annotations", "animal-sniffer-annotations", "perfmark-api", - "proto-google-common-protos") + val validPrefixes = Set("spark-connect", "unused-", "annotations-", + "grpc-", "protobuf-", "gson", "animal-sniffer-annotations", + "perfmark-api", "proto-google-common-protos") cp filterNot { v => validPrefixes.exists(v.data.getName.startsWith) } @@ -809,15 +806,10 @@ object SparkConnect { (assembly / assemblyShadeRules) := Seq( ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.grpc.@1").inAll, - ShadeRule.rename("com.google.common.**" -> "org.sparkproject.connect.guava.@1").inAll, - ShadeRule.rename("com.google.thirdparty.**" -> "org.sparkproject.connect.guava.@1").inAll, ShadeRule.rename("com.google.protobuf.**" -> "org.sparkproject.connect.protobuf.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.android_annotation.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.io_perfmark.@1").inAll, ShadeRule.rename("org.codehaus.mojo.animal_sniffer.**" -> "org.sparkproject.connect.animal_sniffer.@1").inAll, - ShadeRule.rename("com.google.j2objc.annotations.**" -> "org.sparkproject.connect.j2objc_annotations.@1").inAll, - ShadeRule.rename("com.google.errorprone.annotations.**" -> "org.sparkproject.connect.errorprone_annotations.@1").inAll, - ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.checkerframework.@1").inAll, ShadeRule.rename("com.google.gson.**" -> "org.sparkproject.connect.gson.@1").inAll, ShadeRule.rename("com.google.api.**" -> "org.sparkproject.connect.google_protos.api.@1").inAll, ShadeRule.rename("com.google.apps.**" -> "org.sparkproject.connect.google_protos.apps.@1").inAll, @@ -851,7 +843,7 @@ object SparkConnectJdbc { libraryDependencies ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] Seq( "com.google.guava" % "guava" % guavaVersion, "com.google.protobuf" % "protobuf-java" % protoVersion % "protobuf" @@ -860,7 +852,7 @@ object SparkConnectJdbc { dependencyOverrides ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] Seq( "com.google.guava" % "guava" % guavaVersion, "com.google.protobuf" % "protobuf-java" % protoVersion @@ -888,14 +880,17 @@ object SparkConnectJdbc { // Exclude `scala-library` from assembly. (assembly / assemblyPackageScala / assembleArtifact) := false, - // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`,`jsr305-*.jar` and - // `netty-*.jar` and `unused-1.0.0.jar` from assembly. + // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`, `jsr305-*.jar`, + // `error_prone_annotations-*.jar`, `listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar`, + // `j2objc-annotations-*.jar`, `checker-qual-*.jar` and `unused-1.0.0.jar` from assembly. (assembly / assemblyExcludedJars) := { val cp = (assembly / fullClasspath).value cp filter { v => val name = v.data.getName name.startsWith("pmml-model-") || name.startsWith("scala-collection-compat_") || - name.startsWith("jsr305-") || name == "unused-1.0.0.jar" + name.startsWith("jsr305-") || name.startsWith("error_prone_annotations") || + name.startsWith("listenablefuture") || name.startsWith("j2objc-annotations") || + name.startsWith("checker-qual") || name == "unused-1.0.0.jar" } }, // Only include `spark-connect-client-jdbc-*.jar` @@ -912,7 +907,6 @@ object SparkConnectJdbc { ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.client.io.grpc.@1").inAll, ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, - ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll @@ -939,7 +933,7 @@ object SparkConnectClient { libraryDependencies ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] Seq( "com.google.guava" % "guava" % guavaVersion, "com.google.protobuf" % "protobuf-java" % protoVersion % "protobuf" @@ -948,7 +942,7 @@ object SparkConnectClient { dependencyOverrides ++= { val guavaVersion = SbtPomKeys.effectivePom.value.getProperties.get( - "connect.guava.version").asInstanceOf[String] + "guava.version").asInstanceOf[String] Seq( "com.google.guava" % "guava" % guavaVersion, "com.google.protobuf" % "protobuf-java" % protoVersion @@ -976,14 +970,17 @@ object SparkConnectClient { // Exclude `scala-library` from assembly. (assembly / assemblyPackageScala / assembleArtifact) := false, - // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`,`jsr305-*.jar` and - // `netty-*.jar` and `unused-1.0.0.jar` from assembly. + // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`, `jsr305-*.jar`, + // `error_prone_annotations-*.jar`, `listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar`, + // `j2objc-annotations-*.jar`, `checker-qual-*.jar` and `unused-1.0.0.jar` from assembly. (assembly / assemblyExcludedJars) := { val cp = (assembly / fullClasspath).value cp filter { v => val name = v.data.getName name.startsWith("pmml-model-") || name.startsWith("scala-collection-compat_") || - name.startsWith("jsr305-") || name == "unused-1.0.0.jar" + name.startsWith("jsr305-") || name.startsWith("error_prone_annotations") || + name.startsWith("listenablefuture") || name.startsWith("j2objc-annotations") || + name.startsWith("checker-qual") || name == "unused-1.0.0.jar" } }, @@ -991,7 +988,6 @@ object SparkConnectClient { ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.client.io.grpc.@1").inAll, ShadeRule.rename("com.google.**" -> "org.sparkproject.connect.client.com.google.@1").inAll, ShadeRule.rename("io.netty.**" -> "org.sparkproject.connect.client.io.netty.@1").inAll, - ShadeRule.rename("org.checkerframework.**" -> "org.sparkproject.connect.client.org.checkerframework.@1").inAll, ShadeRule.rename("io.perfmark.**" -> "org.sparkproject.connect.client.io.perfmark.@1").inAll, ShadeRule.rename("org.codehaus.**" -> "org.sparkproject.connect.client.org.codehaus.@1").inAll, ShadeRule.rename("android.annotation.**" -> "org.sparkproject.connect.client.android.annotation.@1").inAll diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index 15e3480f2e59d..af34a65f9bdec 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -77,13 +77,11 @@ com.google.guava guava - ${connect.guava.version} compile com.google.guava failureaccess - ${guava.failureaccess.version} compile @@ -177,10 +175,6 @@ io.netty ${spark.shade.packageName}.io.netty - - org.checkerframework - ${spark.shade.packageName}.org.checkerframework - io.perfmark ${spark.shade.packageName}.io.perfmark diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index c5e4e6a1adfda..fb983040843ba 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -77,13 +77,11 @@ com.google.guava guava - ${connect.guava.version} compile com.google.guava failureaccess - ${guava.failureaccess.version} compile com.google.common - ${spark.shade.packageName}.connect.guava - - com.google.common.** - + ${spark.shade.packageName}.guava com.google.thirdparty - ${spark.shade.packageName}.connect.guava - - com.google.thirdparty.** - + ${spark.shade.packageName}.guava.thirdparty + com.google.protobuf ${spark.shade.packageName}.connect.protobuf @@ -344,18 +337,6 @@ org.codehaus.mojo.animal_sniffer ${spark.shade.packageName}.connect.animal_sniffer - - com.google.j2objc.annotations - ${spark.shade.packageName}.connect.j2objc_annotations - - - com.google.errorprone.annotations - ${spark.shade.packageName}.connect.errorprone_annotations - - - org.checkerframework - ${spark.shade.packageName}.connect.checkerframework - com.google.gson ${spark.shade.packageName}.connect.gson From def03c2b9583c71d483d305b5bbba9ade39ed29a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 7 Nov 2025 06:46:16 -0800 Subject: [PATCH 060/400] [SPARK-54226][SQL] Extend Arrow compression to Pandas UDF ### What changes were proposed in this pull request? This is an extension to https://github.com/apache/spark/pull/52747. In https://github.com/apache/spark/pull/52747, we add the support of Arrow compression to `toArrow` and `toPandas` to reduce memory usage. We would like to extend the memory optimization feature to Pandas UDF case. ### Why are the changes needed? To optimize memory usage for Pandas UDF case. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Code v2.0.14 Closes #52925 from viirya/arrow_compress_udf. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun (cherry picked from commit 96ed48db3879fc9e2d250c4548eec4409061d2de) Signed-off-by: Dongjoon Hyun --- .../tests/pandas/test_pandas_grouped_map.py | 60 ++++++++++++++ .../pandas/test_pandas_udf_grouped_agg.py | 36 +++++++++ .../tests/pandas/test_pandas_udf_scalar.py | 56 +++++++++++++ .../execution/arrow/ArrowWriterWrapper.scala | 8 +- .../apache/spark/sql/internal/SQLConf.scala | 15 ++++ .../sql/execution/arrow/ArrowConverters.scala | 6 +- .../python/CoGroupedArrowPythonRunner.scala | 39 ++++++++- .../execution/python/PythonArrowInput.scala | 79 +++++++++++++++++-- 8 files changed, 285 insertions(+), 14 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index ef84673179dcc..b60c5a187fbf7 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -1406,6 +1406,66 @@ def func(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: actual = grouped_df.applyInPandas(func, "value long").collect() self.assertEqual(actual, expected) + def test_grouped_map_pandas_udf_with_compression_codec(self): + # Test grouped map Pandas UDF with different compression codec settings + @pandas_udf("id long, v int, v1 double", PandasUDFType.GROUPED_MAP) + def foo(pdf): + return pdf.assign(v1=pdf.v * pdf.id * 1.0) + + df = self.data + pdf = df.toPandas() + expected = pdf.groupby("id", as_index=False).apply(foo.func).reset_index(drop=True) + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = df.groupby("id").apply(foo).sort("id").toPandas() + assert_frame_equal(expected, result) + + def test_apply_in_pandas_with_compression_codec(self): + # Test applyInPandas with different compression codec settings + def stats(key, pdf): + return pd.DataFrame([(key[0], pdf.v.mean())], columns=["id", "mean"]) + + df = self.data + expected = df.select("id").distinct().withColumn("mean", sf.lit(24.5)).toPandas() + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = ( + df.groupby("id") + .applyInPandas(stats, schema="id long, mean double") + .sort("id") + .toPandas() + ) + assert_frame_equal(expected, result) + + def test_apply_in_pandas_iterator_with_compression_codec(self): + # Test applyInPandas with iterator and compression + df = self.spark.createDataFrame( + [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v") + ) + + def sum_func(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: + total = 0 + for batch in batches: + total += batch["v"].sum() + yield pd.DataFrame({"v": [total]}) + + expected = [Row(v=3.0), Row(v=18.0)] + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = ( + df.groupby("id") + .applyInPandas(sum_func, schema="v double") + .orderBy("v") + .collect() + ) + self.assertEqual(result, expected) + class ApplyInPandasTests(ApplyInPandasTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py index 2b3e42312df99..2958d0e67f1ea 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py @@ -861,6 +861,42 @@ def my_grouped_agg_pandas_udf(x): ], ) + def test_grouped_agg_pandas_udf_with_compression_codec(self): + # Test grouped agg Pandas UDF with different compression codec settings + @pandas_udf("double", PandasUDFType.GROUPED_AGG) + def sum_udf(v): + return v.sum() + + df = self.data + expected = df.groupby("id").agg(sum_udf(df.v)).sort("id").toPandas() + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = df.groupby("id").agg(sum_udf(df.v)).sort("id").toPandas() + assert_frame_equal(expected, result) + + def test_grouped_agg_pandas_udf_with_compression_codec_complex(self): + # Test grouped agg with multiple UDFs and compression + @pandas_udf("double", PandasUDFType.GROUPED_AGG) + def mean_udf(v): + return v.mean() + + @pandas_udf("double", PandasUDFType.GROUPED_AGG) + def sum_udf(v): + return v.sum() + + df = self.data + expected = df.groupby("id").agg(mean_udf(df.v), sum_udf(df.v)).sort("id").toPandas() + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = ( + df.groupby("id").agg(mean_udf(df.v), sum_udf(df.v)).sort("id").toPandas() + ) + assert_frame_equal(expected, result) + class GroupedAggPandasUDFTests(GroupedAggPandasUDFTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py index fbfe1a226b5e2..554c994afc1ed 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py @@ -1988,6 +1988,62 @@ def my_scalar_iter_pandas_udf(it): ], ) + def test_scalar_pandas_udf_with_compression_codec(self): + # Test scalar Pandas UDF with different compression codec settings + @pandas_udf("long") + def plus_one(v): + return v + 1 + + df = self.spark.range(100) + expected = [Row(result=i + 1) for i in range(100)] + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = df.select(plus_one("id").alias("result")).collect() + self.assertEqual(expected, result) + + def test_scalar_pandas_udf_with_compression_codec_complex_types(self): + # Test scalar Pandas UDF with compression for complex types (strings, arrays) + @pandas_udf("string") + def concat_string(v): + return v.apply(lambda x: "value_" + str(x)) + + @pandas_udf(ArrayType(IntegerType())) + def create_array(v): + return v.apply(lambda x: [x, x * 2, x * 3]) + + df = self.spark.range(50) + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + # Test string UDF + result = df.select(concat_string("id").alias("result")).collect() + expected = [Row(result=f"value_{i}") for i in range(50)] + self.assertEqual(expected, result) + + # Test array UDF + result = df.select(create_array("id").alias("result")).collect() + expected = [Row(result=[i, i * 2, i * 3]) for i in range(50)] + self.assertEqual(expected, result) + + def test_scalar_iter_pandas_udf_with_compression_codec(self): + # Test scalar iterator Pandas UDF with compression + @pandas_udf("long", PandasUDFType.SCALAR_ITER) + def plus_two(iterator): + for s in iterator: + yield s + 2 + + df = self.spark.range(100) + expected = [Row(result=i + 2) for i in range(100)] + + for codec in ["none", "zstd", "lz4"]: + with self.subTest(compressionCodec=codec): + with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + result = df.select(plus_two("id").alias("result")).collect() + self.assertEqual(expected, result) + class ScalarPandasUDFTests(ScalarPandasUDFTestsMixin, ReusedSQLTestCase): @classmethod diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriterWrapper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriterWrapper.scala index 6c5799bd241b9..c04bae07f67dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriterWrapper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriterWrapper.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.arrow import java.io.DataOutputStream import org.apache.arrow.memory.BufferAllocator -import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.{VectorSchemaRoot, VectorUnloader} import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.spark.TaskContext @@ -34,6 +34,7 @@ case class ArrowWriterWrapper( var arrowWriter: SparkArrowWriter, var root: VectorSchemaRoot, var allocator: BufferAllocator, + var unloader: VectorUnloader, context: TaskContext) { @volatile var isClosed = false @@ -58,6 +59,7 @@ case class ArrowWriterWrapper( arrowWriter = null root = null allocator = null + unloader = null } } } @@ -77,8 +79,10 @@ object ArrowWriterWrapper { s"stdout writer for $allocatorOwner", 0, Long.MaxValue) val root = VectorSchemaRoot.create(arrowSchema, allocator) val arrowWriter = SparkArrowWriter.create(root) + val streamWriter = new ArrowStreamWriter(root, null, dataOut) streamWriter.start() - ArrowWriterWrapper(streamWriter, arrowWriter, root, allocator, context) + // Unloader will be set by the caller after creation + ArrowWriterWrapper(streamWriter, arrowWriter, root, allocator, null, context) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b8907629ad372..a367971a2fc82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4001,6 +4001,19 @@ object SQLConf { .checkValues(Set("none", "zstd", "lz4")) .createWithDefault("none") + val ARROW_EXECUTION_ZSTD_COMPRESSION_LEVEL = + buildConf("spark.sql.execution.arrow.zstd.compressionLevel") + .doc("Compression level for Zstandard (zstd) codec when compressing Arrow IPC data. " + + "This config is only used when spark.sql.execution.arrow.compressionCodec is set to " + + "'zstd'. Valid values are integers from 1 (fastest, lowest compression) to 22 " + + "(slowest, highest compression). The default value 3 provides a good balance between " + + "compression speed and compression ratio.") + .version("4.1.0") + .intConf + .checkValue(level => level >= 1 && level <= 22, + "Zstd compression level must be between 1 and 22") + .createWithDefault(3) + val ARROW_TRANSFORM_WITH_STATE_IN_PYSPARK_MAX_STATE_RECORDS_PER_BATCH = buildConf("spark.sql.execution.arrow.transformWithStateInPySpark.maxStateRecordsPerBatch") .doc("When using TransformWithState in PySpark (both Python Row and Pandas), limit " + @@ -7348,6 +7361,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def arrowCompressionCodec: String = getConf(ARROW_EXECUTION_COMPRESSION_CODEC) + def arrowZstdCompressionLevel: Int = getConf(ARROW_EXECUTION_ZSTD_COMPRESSION_LEVEL) + def arrowTransformWithStateInPySparkMaxStateRecordsPerBatch: Int = getConf(ARROW_TRANSFORM_WITH_STATE_IN_PYSPARK_MAX_STATE_RECORDS_PER_BATCH) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 7f260bd2efd04..8b031af14e8b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -31,6 +31,7 @@ import org.apache.arrow.vector.compression.{CompressionCodec, NoCompressionCodec import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter, ReadChannel, WriteChannel} import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, IpcOption, MessageSerializer} +import org.apache.spark.SparkException import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils @@ -102,15 +103,16 @@ private[sql] object ArrowConverters extends Logging { private val codec = compressionCodecName match { case "none" => NoCompressionCodec.INSTANCE case "zstd" => + val compressionLevel = SQLConf.get.arrowZstdCompressionLevel val factory = CompressionCodec.Factory.INSTANCE - val codecType = new ZstdCompressionCodec().getCodecType() + val codecType = new ZstdCompressionCodec(compressionLevel).getCodecType() factory.createCodec(codecType) case "lz4" => val factory = CompressionCodec.Factory.INSTANCE val codecType = new Lz4CompressionCodec().getCodecType() factory.createCodec(codecType) case other => - throw new IllegalArgumentException( + throw SparkException.internalError( s"Unsupported Arrow compression codec: $other. Supported values: none, zstd, lz4") } protected val unloader = new VectorUnloader(root, true, codec, true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala index 00eb9039d05cf..50013e5338199 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala @@ -20,7 +20,11 @@ package org.apache.spark.sql.execution.python import java.io.DataOutputStream import java.util -import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.arrow.compression.{Lz4CompressionCodec, ZstdCompressionCodec} +import org.apache.arrow.vector.{VectorSchemaRoot, VectorUnloader} +import org.apache.arrow.vector.compression.{CompressionCodec, NoCompressionCodec} + +import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, PythonRDD, PythonWorker} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.arrow.ArrowWriterWrapper @@ -76,6 +80,27 @@ class CoGroupedArrowPythonRunner( if (v > 0) v else Int.MaxValue } private val maxBytesPerBatch: Long = SQLConf.get.arrowMaxBytesPerBatch + private val compressionCodecName: String = SQLConf.get.arrowCompressionCodec + + // Helper method to create VectorUnloader with compression + private def createUnloader(root: VectorSchemaRoot): VectorUnloader = { + val codec = compressionCodecName match { + case "none" => NoCompressionCodec.INSTANCE + case "zstd" => + val compressionLevel = SQLConf.get.arrowZstdCompressionLevel + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new ZstdCompressionCodec(compressionLevel).getCodecType() + factory.createCodec(codecType) + case "lz4" => + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new Lz4CompressionCodec().getCodecType() + factory.createCodec(codecType) + case other => + throw SparkException.internalError( + s"Unsupported Arrow compression codec: $other. Supported values: none, zstd, lz4") + } + new VectorUnloader(root, true, codec, true) + } protected def newWriter( env: SparkEnv, @@ -136,13 +161,17 @@ class CoGroupedArrowPythonRunner( leftGroupArrowWriter = ArrowWriterWrapper.createAndStartArrowWriter(leftSchema, timeZoneId, pythonExec + " (left)", errorOnDuplicatedFieldNames = true, largeVarTypes, dataOut, context) + // Set the unloader with compression after creating the writer + leftGroupArrowWriter.unloader = createUnloader(leftGroupArrowWriter.root) } numRowsInBatch = BatchedPythonArrowInput.writeSizedBatch( leftGroupArrowWriter.arrowWriter, leftGroupArrowWriter.streamWriter, nextBatchInLeftGroup, maxBytesPerBatch, - maxRecordsPerBatch) + maxRecordsPerBatch, + leftGroupArrowWriter.unloader, + dataOut) if (!nextBatchInLeftGroup.hasNext) { leftGroupArrowWriter.streamWriter.end() @@ -155,13 +184,17 @@ class CoGroupedArrowPythonRunner( rightGroupArrowWriter = ArrowWriterWrapper.createAndStartArrowWriter(rightSchema, timeZoneId, pythonExec + " (right)", errorOnDuplicatedFieldNames = true, largeVarTypes, dataOut, context) + // Set the unloader with compression after creating the writer + rightGroupArrowWriter.unloader = createUnloader(rightGroupArrowWriter.root) } numRowsInBatch = BatchedPythonArrowInput.writeSizedBatch( rightGroupArrowWriter.arrowWriter, rightGroupArrowWriter.streamWriter, nextBatchInRightGroup, maxBytesPerBatch, - maxRecordsPerBatch) + maxRecordsPerBatch, + rightGroupArrowWriter.unloader, + dataOut) if (!nextBatchInRightGroup.hasNext) { rightGroupArrowWriter.streamWriter.end() rightGroupArrowWriter.close() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala index b2ec96c5b29f8..f77b0a9342b02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala @@ -17,11 +17,16 @@ package org.apache.spark.sql.execution.python import java.io.DataOutputStream +import java.nio.channels.Channels -import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.compression.{Lz4CompressionCodec, ZstdCompressionCodec} +import org.apache.arrow.vector.{VectorSchemaRoot, VectorUnloader} +import org.apache.arrow.vector.compression.{CompressionCodec, NoCompressionCodec} import org.apache.arrow.vector.ipc.ArrowStreamWriter +import org.apache.arrow.vector.ipc.WriteChannel +import org.apache.arrow.vector.ipc.message.MessageSerializer -import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.api.python.{BasePythonRunner, PythonRDD, PythonWorker} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.arrow @@ -70,6 +75,26 @@ private[python] trait PythonArrowInput[IN] { self: BasePythonRunner[IN, _] => protected val allocator = ArrowUtils.rootAllocator.newChildAllocator(s"stdout writer for $pythonExec", 0, Long.MaxValue) protected val root = VectorSchemaRoot.create(arrowSchema, allocator) + + // Create compression codec based on config + private val compressionCodecName = SQLConf.get.arrowCompressionCodec + private val codec = compressionCodecName match { + case "none" => NoCompressionCodec.INSTANCE + case "zstd" => + val compressionLevel = SQLConf.get.arrowZstdCompressionLevel + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new ZstdCompressionCodec(compressionLevel).getCodecType() + factory.createCodec(codecType) + case "lz4" => + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new Lz4CompressionCodec().getCodecType() + factory.createCodec(codecType) + case other => + throw SparkException.internalError( + s"Unsupported Arrow compression codec: $other. Supported values: none, zstd, lz4") + } + protected val unloader = new VectorUnloader(root, true, codec, true) + protected var writer: ArrowStreamWriter = _ protected def close(): Unit = { @@ -137,7 +162,14 @@ private[python] trait BasicPythonArrowInput extends PythonArrowInput[Iterator[In } arrowWriter.finish() - writer.writeBatch() + // Use unloader to get compressed batch and write it manually + val batch = unloader.getRecordBatch() + try { + val writeChannel = new WriteChannel(Channels.newChannel(dataOut)) + MessageSerializer.serialize(writeChannel, batch) + } finally { + batch.close() + } arrowWriter.reset() val deltaData = dataOut.size() - startData pythonMetrics("pythonDataSent") += deltaData @@ -169,7 +201,8 @@ private[python] trait BatchedPythonArrowInput extends BasicPythonArrowInput { val startData = dataOut.size() val numRowsInBatch = BatchedPythonArrowInput.writeSizedBatch( - arrowWriter, writer, nextBatchStart, maxBytesPerBatch, maxRecordsPerBatch) + arrowWriter, writer, nextBatchStart, maxBytesPerBatch, maxRecordsPerBatch, unloader, + dataOut) assert(0 < numRowsInBatch && numRowsInBatch <= maxRecordsPerBatch, numRowsInBatch) val deltaData = dataOut.size() - startData @@ -209,7 +242,9 @@ private[python] object BatchedPythonArrowInput { writer: ArrowStreamWriter, rowIter: Iterator[InternalRow], maxBytesPerBatch: Long, - maxRecordsPerBatch: Int): Int = { + maxRecordsPerBatch: Int, + unloader: VectorUnloader, + dataOut: DataOutputStream): Int = { var numRowsInBatch: Int = 0 def underBatchSizeLimit: Boolean = @@ -221,7 +256,14 @@ private[python] object BatchedPythonArrowInput { numRowsInBatch += 1 } arrowWriter.finish() - writer.writeBatch() + // Use unloader to get compressed batch and write it manually + val batch = unloader.getRecordBatch() + try { + val writeChannel = new WriteChannel(Channels.newChannel(dataOut)) + MessageSerializer.serialize(writeChannel, batch) + } finally { + batch.close() + } arrowWriter.reset() numRowsInBatch } @@ -231,6 +273,26 @@ private[python] object BatchedPythonArrowInput { * Enables an optimization that splits each group into the sized batches. */ private[python] trait GroupedPythonArrowInput { self: RowInputArrowPythonRunner => + + // Helper method to create VectorUnloader with compression for grouped operations + private def createUnloaderForGroup(root: VectorSchemaRoot): VectorUnloader = { + val codec = SQLConf.get.arrowCompressionCodec match { + case "none" => NoCompressionCodec.INSTANCE + case "zstd" => + val compressionLevel = SQLConf.get.arrowZstdCompressionLevel + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new ZstdCompressionCodec(compressionLevel).getCodecType() + factory.createCodec(codecType) + case "lz4" => + val factory = CompressionCodec.Factory.INSTANCE + val codecType = new Lz4CompressionCodec().getCodecType() + factory.createCodec(codecType) + case other => + throw SparkException.internalError( + s"Unsupported Arrow compression codec: $other. Supported values: none, zstd, lz4") + } + new VectorUnloader(root, true, codec, true) + } protected override def newWriter( env: SparkEnv, worker: PythonWorker, @@ -255,13 +317,16 @@ private[python] trait GroupedPythonArrowInput { self: RowInputArrowPythonRunner writer = ArrowWriterWrapper.createAndStartArrowWriter( schema, timeZoneId, pythonExec, errorOnDuplicatedFieldNames, largeVarTypes, dataOut, context) + // Set the unloader with compression after creating the writer + writer.unloader = createUnloaderForGroup(writer.root) nextBatchStart = inputIterator.next() } } if (nextBatchStart.hasNext) { val startData = dataOut.size() val numRowsInBatch: Int = BatchedPythonArrowInput.writeSizedBatch(writer.arrowWriter, - writer.streamWriter, nextBatchStart, maxBytesPerBatch, maxRecordsPerBatch) + writer.streamWriter, nextBatchStart, maxBytesPerBatch, maxRecordsPerBatch, + writer.unloader, dataOut) if (!nextBatchStart.hasNext) { writer.streamWriter.end() // We don't need a try catch block here as the close() method is registered with From fd0828852bfdc7201259e7e823ff83b46180d6e1 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Fri, 7 Nov 2025 06:48:31 -0800 Subject: [PATCH 061/400] [SPARK-54202][GEO][SQL] Allow casting from GeometryType(srid) to GeometryType(ANY) ### What changes were proposed in this pull request? This PR allows casting fixed SRID type `GEOMETRY()` to mixed SRID type `GEOMETRY(ANY)`. ### Why are the changes needed? Enable explicit casting between geometry types. ### Does this PR introduce _any_ user-facing change? Yes, casting `GEOMETRY()` to `GEOMETRY(ANY)` is now allowed. ### How was this patch tested? Added new unit tests: - `StUtilsSuite` - `CastSuiteBase` Added new e2e SQL tests: - `st-functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52904 from uros-db/geo-cast-geom_any. Authored-by: Uros Bojanic Signed-off-by: Dongjoon Hyun (cherry picked from commit 7bc3e75c03e9001aac20f0ae1f00eee856b99c83) Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/Cast.scala | 11 +++++ .../catalyst/expressions/CastSuiteBase.scala | 23 +++++++++++ .../nonansi/st-functions.sql.out | 29 +++++++++++++ .../analyzer-results/st-functions.sql.out | 29 +++++++++++++ .../sql-tests/inputs/st-functions.sql | 5 +++ .../results/nonansi/st-functions.sql.out | 32 +++++++++++++++ .../sql-tests/results/st-functions.sql.out | 32 +++++++++++++++ .../apache/spark/sql/STExpressionsSuite.scala | 41 +++++++++++++++++++ 8 files changed, 202 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 22f1ad73a0e84..10f4c5c00f043 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -170,6 +170,9 @@ object Cast extends QueryErrorsBase { // Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed. case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid => true + // Casts from concrete GEOMETRY(srid) to mixed GEOMETRY(ANY) is allowed. + case (gt1: GeometryType, gt2: GeometryType) if !gt1.isMixedSrid && gt2.isMixedSrid => + true case _ => false } @@ -303,6 +306,9 @@ object Cast extends QueryErrorsBase { // Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed. case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid => true + // Casts from concrete GEOMETRY(srid) to mixed GEOMETRY(ANY) is allowed. + case (gt1: GeometryType, gt2: GeometryType) if !gt1.isMixedSrid && gt2.isMixedSrid => + true case _ => false } @@ -1157,6 +1163,8 @@ case class Cast( private[this] def castToGeometry(from: DataType): Any => Any = from match { case _: GeographyType => buildCast[GeographyVal](_, STUtils.geographyToGeometry) + case _: GeometryType => + identity } private[this] def castArray(fromType: DataType, toType: DataType): Any => Any = { @@ -2201,6 +2209,9 @@ case class Cast( case _: GeographyType => (c, evPrim, _) => code"$evPrim = org.apache.spark.sql.catalyst.util.STUtils.geographyToGeometry($c);" + case _: GeometryType => + (c, evPrim, _) => + code"$evPrim = $c;" } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index 2220f56255f0a..e18a489d36f3b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -1544,6 +1544,29 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } } + test("Casting GeometryType to GeometryType") { + // Casting from fixed SRID GEOMETRY() to mixed SRID GEOMETRY(ANY) is always allowed. + // Type casting is always safe in this direction, so no additional constraints are imposed. + // Casting from mixed SRID GEOMETRY(ANY) to fixed SRID GEOMETRY() is not allowed. + // Type casting can be unsafe in this direction, because per-row SRID values may be different. + + // Valid cast test cases. + val canCastTestCases: Seq[(DataType, DataType)] = Seq( + (GeometryType(0), GeometryType("ANY")), + (GeometryType(3857), GeometryType("ANY")), + (GeometryType(4326), GeometryType("ANY")) + ) + // Iterate over the test cases and verify casting. + canCastTestCases.foreach { case (fromType, toType) => + // Cast can be performed from `fromType` to `toType`. + assert(Cast.canCast(fromType, toType)) + assert(Cast.canAnsiCast(fromType, toType)) + // Cast cannot be performed from `toType` to `fromType`. + assert(!Cast.canCast(toType, fromType)) + assert(!Cast.canAnsiCast(toType, fromType)) + } + } + test("cast string to time") { checkEvaluation(cast(Literal.create("0:0:0"), TimeType()), 0L) checkEvaluation(cast(Literal.create(" 01:2:3.01 "), TimeType(2)), localTime(1, 2, 3, 10000)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out index 1d2094f3b9ef1..a564c6a329327 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out @@ -124,6 +124,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)))) AS result +-- !query analysis +Project [hex(st_asbinary(cast(st_geomfromwkb(0x0101000000000000000000F03F0000000000000040) as geometry(any)))) AS result#x] ++- OneRowRelation + + +-- !query +SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY)) AS GEOMETRY(4326))\"", + "srcType" : "\"GEOMETRY(ANY)\"", + "targetType" : "\"GEOMETRY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 107, + "fragment" : "CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out index 1d2094f3b9ef1..a564c6a329327 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out @@ -124,6 +124,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)))) AS result +-- !query analysis +Project [hex(st_asbinary(cast(st_geomfromwkb(0x0101000000000000000000F03F0000000000000040) as geometry(any)))) AS result#x] ++- OneRowRelation + + +-- !query +SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY)) AS GEOMETRY(4326))\"", + "srcType" : "\"GEOMETRY(ANY)\"", + "targetType" : "\"GEOMETRY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 107, + "fragment" : "CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql index 7f4b77c7e0f9c..ceda713983050 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql @@ -23,6 +23,11 @@ SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f000000000 -- Error handling: mismatched SRIDs. SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)) AS result; +-- Casting GEOMETRY() to GEOMETRY(ANY) is allowed. +SELECT hex(ST_AsBinary(CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)))) AS result; +-- Casting GEOMETRY(ANY) to GEOMETRY() is not allowed. +SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326)) AS result; + ---- ST reader/writer expressions -- WKB (Well-Known Binary) round-trip tests for GEOGRAPHY and GEOMETRY types. diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out index 088c49cb030a3..c01534c64e7c6 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out @@ -137,6 +137,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)))) AS result +-- !query schema +struct +-- !query output +0101000000000000000000F03F0000000000000040 + + +-- !query +SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY)) AS GEOMETRY(4326))\"", + "srcType" : "\"GEOMETRY(ANY)\"", + "targetType" : "\"GEOMETRY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 107, + "fragment" : "CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out index 088c49cb030a3..c01534c64e7c6 100644 --- a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out @@ -137,6 +137,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT hex(ST_AsBinary(CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(ANY)))) AS result +-- !query schema +struct +-- !query output +0101000000000000000000F03F0000000000000040 + + +-- !query +SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"CAST(CAST(st_geomfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOMETRY(ANY)) AS GEOMETRY(4326))\"", + "srcType" : "\"GEOMETRY(ANY)\"", + "targetType" : "\"GEOMETRY(4326)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 107, + "fragment" : "CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326))" + } ] +} + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 4c34a5c8f78f1..79e7ecbdf4c38 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -34,6 +34,7 @@ class STExpressionsSuite private final val mixedSridGeographyType: DataType = GeographyType("ANY") private final val defaultGeometrySrid: Int = ExpressionDefaults.DEFAULT_GEOMETRY_SRID private final val defaultGeometryType: DataType = GeometryType(defaultGeometrySrid) + private final val mixedSridGeometryType: DataType = GeometryType("ANY") // Private helper method to assert the data type of a query result. private def assertType(query: String, expectedDataType: DataType) = { @@ -82,6 +83,46 @@ class STExpressionsSuite } } + test("Cast GEOMETRY(srid) to GEOMETRY(ANY)") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + val wkb = Hex.unhex(wkbString.getBytes()) + val wkbLiteral = Literal.create(wkb, BinaryType) + + // Construct the input GEOMETRY expression. + val geomExpr = ST_GeomFromWKB(wkbLiteral) + assert(geomExpr.dataType.sameType(defaultGeometryType)) + checkEvaluation(ST_AsBinary(geomExpr), wkb) + // Cast the GEOMETRY with fixed SRID to GEOMETRY with mixed SRID. + val castExpr = Cast(geomExpr, mixedSridGeometryType) + assert(castExpr.dataType.sameType(mixedSridGeometryType)) + checkEvaluation(ST_AsBinary(castExpr), wkb) + + // Construct the input GEOMETRY SQL query, using WKB literal. + val geomQueryLit: String = s"ST_GeomFromWKB(X'$wkbString')" + assertType(s"SELECT $geomQueryLit", defaultGeometryType) + checkAnswer(sql(s"SELECT ST_AsBinary($geomQueryLit)"), Row(wkb)) + // Cast the GEOMETRY with fixed SRID to GEOMETRY with mixed SRID. + val castQueryLit = s"$geomQueryLit::GEOMETRY(ANY)" + assertType(s"SELECT $castQueryLit", mixedSridGeometryType) + checkAnswer(sql(s"SELECT ST_AsBinary($castQueryLit)"), Row(wkb)) + + withTable("tbl") { + // Construct the test table with WKB. + sql(s"CREATE TABLE tbl (wkb BINARY)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString')") + + // Construct the input GEOMETRY SQL query, using WKB column. + val geomQueryCol: String = s"ST_GeomFromWKB(wkb)" + assertType(s"SELECT $geomQueryCol FROM tbl", defaultGeometryType) + checkAnswer(sql(s"SELECT ST_AsBinary($geomQueryCol) FROM tbl"), Row(wkb)) + // Cast the GEOMETRY with fixed SRID to GEOMETRY with mixed SRID. + val castQueryCol = s"$geomQueryCol::GEOMETRY(ANY)" + assertType(s"SELECT $castQueryCol FROM tbl", mixedSridGeometryType) + checkAnswer(sql(s"SELECT ST_AsBinary($castQueryCol) FROM tbl"), Row(wkb)) + } + } + /** ST reader/writer expressions. */ test("ST_AsBinary") { From 5b372d8ab2146f329e2956829821dfdde9ce3d9d Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 7 Nov 2025 08:15:56 -0800 Subject: [PATCH 062/400] [SPARK-54234][PYTHON][CONNECT] Not need to attach PlanId in grouping column names in df.groupBy ### What changes were proposed in this pull request? Not need to attach PlanId in grouping column names in df.groupBy ### Why are the changes needed? to be more consistent with classic mode https://github.com/apache/spark/blob/e75ca577923f9f465eb06b4df814c00143fa41ea/sql/api/src/main/scala/org/apache/spark/sql/Dataset.scala#L1318-L1320 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #52933 from zhengruifeng/connect_group_key_relax. Authored-by: Ruifeng Zheng Signed-off-by: Wenchen Fan (cherry picked from commit 37997cef6dfff7a0c093f75e720d6792e1eafefc) Signed-off-by: Wenchen Fan --- python/pyspark/sql/connect/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 71a499afd2ff7..862974f111658 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -587,7 +587,7 @@ def groupBy(self, *cols: "ColumnOrNameOrOrdinal") -> "GroupedData": if isinstance(c, Column): _cols.append(c) elif isinstance(c, str): - _cols.append(self[c]) + _cols.append(F.col(c)) elif isinstance(c, int) and not isinstance(c, bool): if c < 1: raise PySparkIndexError( From b6999e0fd547b0be0d507ab9d8cd7cb4f0a67062 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 7 Nov 2025 09:20:39 -0800 Subject: [PATCH 063/400] [SPARK-54241][INFRA] Enable `NOLINT_ON_COMPILE` for all PySpark GitHub Action jobs ### What changes were proposed in this pull request? This PR aims to optimize PySpark GitHub Action jobs by enabling `NOLINT_ON_COMPILE`. ### Why are the changes needed? Since we have a separate `linter` job, we don't need to run at every PySpark test pipelines. Although it's a small part per job, it will help us collectively because almost 50% of our CIs are Python CIs. In addition, this will improve the community PRs. - https://github.com/apache/spark/actions/workflows/build_python_pypy3.10.yml - https://github.com/apache/spark/actions/workflows/build_python_3.10.yml - https://github.com/apache/spark/actions/workflows/build_python_3.11_classic_only.yml - https://github.com/apache/spark/actions/workflows/build_python_3.11_arm.yml - https://github.com/apache/spark/actions/workflows/build_python_3.11_macos.yml - https://github.com/apache/spark/actions/workflows/build_python_3.11_macos26.yml - https://github.com/apache/spark/actions/workflows/build_python_numpy_2.1.3.yml - https://github.com/apache/spark/actions/workflows/build_python_3.12.yml - https://github.com/apache/spark/actions/workflows/build_python_3.13.yml - https://github.com/apache/spark/actions/workflows/build_python_3.13_nogil.yml - https://github.com/apache/spark/actions/workflows/build_python_3.14.yml - https://github.com/apache/spark/actions/workflows/build_python_minimum.yml - https://github.com/apache/spark/actions/workflows/build_python_ps_minimum.yml - https://github.com/apache/spark/actions/workflows/build_python_connect35.yml - https://github.com/apache/spark/actions/workflows/build_python_connect.yml - https://github.com/apache/spark/actions/workflows/build_branch41_python.yml - https://github.com/apache/spark/actions/workflows/build_branch41_python_pypy3.10.yml - https://github.com/apache/spark/actions/workflows/build_branch40_python.yml - https://github.com/apache/spark/actions/workflows/build_branch40_python_pypy3.10.yml - https://github.com/apache/spark/actions/workflows/build_branch35_python.yml ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52942 from dongjoon-hyun/SPARK-54241. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 050d3978d332a91a97ba63fc33fe6506bce1fe49) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 2609f84d98cc3..d0682cc92414e 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -547,6 +547,7 @@ jobs: HIVE_PROFILE: hive2.3 GITHUB_PREV_SHA: ${{ github.event.before }} SPARK_LOCAL_IP: localhost + NOLINT_ON_COMPILE: true SKIP_UNIDOC: true SKIP_MIMA: true SKIP_PACKAGING: true From d9d20ceae033b446daf6c68a439f49183b749b88 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 7 Nov 2025 09:59:32 -0800 Subject: [PATCH 064/400] [SPARK-54242][BUILD] Skip `Checkstyle` if `NOLINT_ON_COMPILE` is true ### What changes were proposed in this pull request? This PR aims to skip `Checkstyle` if `NOLINT_ON_COMPILE` is true. ### Why are the changes needed? Like `Scalastyle`, `Checkstyle` is also a kind of linting. So, we had better skip it when a user set `NOLINT_ON_COMPILE` to `true`. ### Does this PR introduce _any_ user-facing change? No Spark behavior change because this is only a build change. ### How was this patch tested? Pass the CIs and manually. ``` $ NOLINT_ON_COMPILE=true build/sbt package | grep checkstyle ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52943 from dongjoon-hyun/SPARK-54242. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 7f81bd7436feb31ed6e4c61bc71ad8580fa5ad66) Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 90c6b6c844a9f..6b9bc8df5ce41 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -392,7 +392,8 @@ object SparkBuild extends PomBuild { /* Enable shared settings on all projects */ (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ copyJarsProjects ++ Seq(spark, tools)) .foreach(enable(sharedSettings ++ DependencyOverrides.settings ++ - ExcludedDependencies.settings ++ Checkstyle.settings ++ ExcludeShims.settings)) + ExcludedDependencies.settings ++ (if (noLintOnCompile) Nil else Checkstyle.settings) ++ + ExcludeShims.settings)) /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) From 1e2c2d1921fca3e683831784de7fe519e6ac55ae Mon Sep 17 00:00:00 2001 From: Ziya Mukhtarov Date: Fri, 7 Nov 2025 10:29:52 -0800 Subject: [PATCH 065/400] [SPARK-54220][SQL] NullType/VOID/UNKNOWN Type Support in Parquet ### What changes were proposed in this pull request? This PR adds support for reading/writing NullType columns in Parquet files via the `UNKNOWN` logical type annotation. Notable changes are: - Changing `ParquetFileFormat.supportDataType` to support NullType - Changing `ParquetToSparkSchemaConverter` to infer NullType if a primitive type has `UNKNOWN` type annotation and there's no Spark-provided expected type - Changing `SparkToParquetSchemaConverter` to convert NullType into a Parquet Boolean physical type with `UNKNOWN` annotation (physical type selection here is arbitrary) - Optimization in `On/OffHeapColumnVector` to not allocate memory if whole vector is guaranteed to hold only nulls, which is the case for NullType columns and columns that are missing from the file. ### Why are the changes needed? To support reading/writing NullType columns in Parquet files. ### Does this PR introduce _any_ user-facing change? Yes. Previously, trying to read or write Parquet files where the schema contained NullType column would throw an error. Now, we no longer throw, and instead write/read data as expected, using the `UNKNOWN` type annotation. ### How was this patch tested? Manually verified that the Parquet files we write is readable by Apache Arrow. Also verified that we can read a simple Parquet file with NullType written by Apache Arrow. Added a new unit test. Fixed existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52922 from ZiyaZa/parquet-unknown. Authored-by: Ziya Mukhtarov Signed-off-by: Wenchen Fan (cherry picked from commit c04e095779f4c9d02152d1db3a510ee21905688f) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 10 ++++ .../parquet/ParquetColumnVector.java | 6 +- .../parquet/ParquetVectorUpdaterFactory.java | 44 +++++++++++++- .../vectorized/OffHeapColumnVector.java | 6 +- .../vectorized/OnHeapColumnVector.java | 6 +- .../vectorized/WritableColumnVector.java | 25 +++++--- .../parquet/ParquetFileFormat.scala | 2 +- .../parquet/ParquetReadSupport.scala | 4 +- .../parquet/ParquetRowConverter.scala | 7 +++ .../parquet/ParquetSchemaConverter.scala | 8 ++- .../datasources/parquet/ParquetUtils.scala | 4 +- .../parquet/ParquetWriteSupport.scala | 5 +- .../spark/sql/FileBasedDataSourceSuite.scala | 6 +- .../datasources/parquet/ParquetIOSuite.scala | 57 +++++++++++++++++++ .../parquet/ParquetSchemaSuite.scala | 34 +++++++++++ .../sql/hive/execution/HiveDDLSuite.scala | 23 ++------ 16 files changed, 205 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a367971a2fc82..366c5700e06f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1552,6 +1552,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PARQUET_VECTORIZED_READER_NULL_TYPE_ENABLED = + buildConf("spark.sql.parquet.enableNullTypeVectorizedReader") + .doc("Enables vectorized Parquet reader support for NullType columns.") + .version("4.1.0") + .booleanConf + .createWithDefault(true) + val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled") .doc("If true, enables Parquet's native record-level filtering using the pushed down " + "filters. " + @@ -6839,6 +6846,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def parquetVectorizedReaderNestedColumnEnabled: Boolean = getConf(PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED) + def parquetVectorizedReaderNullTypeEnabled: Boolean = + getConf(PARQUET_VECTORIZED_READER_NULL_TYPE_ENABLED) + def parquetVectorizedReaderBatchSize: Int = getConf(PARQUET_VECTORIZED_READER_BATCH_SIZE) def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java index 37c936c84d5f7..002b7569a6e09 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java @@ -80,7 +80,7 @@ final class ParquetColumnVector { } if (defaultValue == null) { - vector.setAllNull(); + vector.setMissing(); return; } // For Parquet tables whose columns have associated DEFAULT values, this reader must return @@ -137,7 +137,7 @@ final class ParquetColumnVector { // Only use levels from non-missing child, this can happen if only some but not all // fields of a struct are missing. - if (!childCv.vector.isAllNull()) { + if (!childCv.vector.isMissing()) { allChildrenAreMissing = false; this.repetitionLevels = childCv.repetitionLevels; this.definitionLevels = childCv.definitionLevels; @@ -147,7 +147,7 @@ final class ParquetColumnVector { // This can happen if all the fields of a struct are missing, in which case we should mark // the struct itself as a missing column if (allChildrenAreMissing) { - vector.setAllNull(); + vector.setMissing(); } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java index eb6c84b8113b8..4f90f878da86a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java @@ -26,6 +26,7 @@ import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; import org.apache.parquet.schema.LogicalTypeAnnotation.TimeLogicalTypeAnnotation; import org.apache.parquet.schema.LogicalTypeAnnotation.TimestampLogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.UnknownLogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; import org.apache.spark.SparkUnsupportedOperationException; @@ -70,7 +71,12 @@ public class ParquetVectorUpdaterFactory { } public ParquetVectorUpdater getUpdater(ColumnDescriptor descriptor, DataType sparkType) { - PrimitiveType.PrimitiveTypeName typeName = descriptor.getPrimitiveType().getPrimitiveTypeName(); + PrimitiveType type = descriptor.getPrimitiveType(); + PrimitiveType.PrimitiveTypeName typeName = type.getPrimitiveTypeName(); + boolean isUnknownType = type.getLogicalTypeAnnotation() instanceof UnknownLogicalTypeAnnotation; + if (isUnknownType && sparkType instanceof NullType) { + return new NullTypeUpdater(); + } switch (typeName) { case BOOLEAN -> { @@ -244,6 +250,42 @@ boolean isUnsignedIntTypeMatched(int bitWidth) { !annotation.isSigned() && annotation.getBitWidth() == bitWidth; } + /** + * Updater should not be called if all values are nulls, so all methods throw exception here. + */ + private static class NullTypeUpdater implements ParquetVectorUpdater { + @Override + public void readValues( + int total, + int offset, + WritableColumnVector values, + VectorizedValuesReader valuesReader) { + throw SparkUnsupportedOperationException.apply(); + } + + @Override + public void skipValues(int total, VectorizedValuesReader valuesReader) { + throw SparkUnsupportedOperationException.apply(); + } + + @Override + public void readValue( + int offset, + WritableColumnVector values, + VectorizedValuesReader valuesReader) { + throw SparkUnsupportedOperationException.apply(); + } + + @Override + public void decodeSingleDictionaryId( + int offset, + WritableColumnVector values, + WritableColumnVector dictionaryIds, + Dictionary dictionary) { + throw SparkUnsupportedOperationException.apply(); + } + } + private static class BooleanUpdater implements ParquetVectorUpdater { @Override public void readValues( diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 2f64ffb42aa06..42454b283d098 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -111,12 +111,14 @@ public void putNotNull(int rowId) { @Override public void putNull(int rowId) { + if (isAllNull()) return; // Skip writing nulls to all-null vector. Platform.putByte(null, nulls + rowId, (byte) 1); ++numNulls; } @Override public void putNulls(int rowId, int count) { + if (isAllNull()) return; // Skip writing nulls to all-null vector. long offset = nulls + rowId; for (int i = 0; i < count; ++i, ++offset) { Platform.putByte(null, offset, (byte) 1); @@ -135,7 +137,7 @@ public void putNotNulls(int rowId, int count) { @Override public boolean isNullAt(int rowId) { - return isAllNull || Platform.getByte(null, nulls + rowId) == 1; + return isAllNull() || Platform.getByte(null, nulls + rowId) == 1; } // @@ -603,6 +605,8 @@ public int putByteArray(int rowId, byte[] value, int offset, int length) { // Split out the slow path. @Override protected void reserveInternal(int newCapacity) { + if (isAllNull()) return; // Skip allocation for all-null vector. + int oldCapacity = (nulls == 0L) ? 0 : capacity; if (isArray() || type instanceof MapType) { this.lengthData = diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index cd8d0b688bedb..401e499fee300 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -108,12 +108,14 @@ public void putNotNull(int rowId) { @Override public void putNull(int rowId) { + if (isAllNull()) return; // Skip writing nulls to all-null vector. nulls[rowId] = (byte)1; ++numNulls; } @Override public void putNulls(int rowId, int count) { + if (isAllNull()) return; // Skip writing nulls to all-null vector. for (int i = 0; i < count; ++i) { nulls[rowId + i] = (byte)1; } @@ -130,7 +132,7 @@ public void putNotNulls(int rowId, int count) { @Override public boolean isNullAt(int rowId) { - return isAllNull || nulls[rowId] == 1; + return isAllNull() || nulls[rowId] == 1; } // @@ -577,6 +579,8 @@ public int putByteArray(int rowId, byte[] value, int offset, int length) { // Spilt this function out since it is the slow path. @Override protected void reserveInternal(int newCapacity) { + if (isAllNull()) return; // Skip allocation for all-null vector. + if (isArray() || type instanceof MapType) { int[] newLengths = new int[newCapacity]; int[] newOffsets = new int[newCapacity]; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index 3f552679bb6f1..c4f06e07911d3 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -59,7 +59,7 @@ public abstract class WritableColumnVector extends ColumnVector { * Resets this column for writing. The currently stored values are no longer accessible. */ public void reset() { - if (isConstant || isAllNull) return; + if (isConstant || isAllNull()) return; if (childColumns != null) { for (WritableColumnVector c: childColumns) { @@ -142,7 +142,7 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { @Override public boolean hasNull() { - return isAllNull || numNulls > 0; + return isAllNull() || numNulls > 0; } @Override @@ -876,17 +876,24 @@ public final void setIsConstant() { } /** - * Marks this column only contains null values. + * Marks this column missing from the file. */ - public final void setAllNull() { - isAllNull = true; + public final void setMissing() { + isMissing = true; + } + + /** + * Whether this column is missing from the file. + */ + public final boolean isMissing() { + return isMissing; } /** * Whether this column only contains null values. */ public final boolean isAllNull() { - return isAllNull; + return isMissing || type instanceof NullType; } /** @@ -921,10 +928,10 @@ public final boolean isAllNull() { protected boolean isConstant; /** - * True if this column only contains nulls. This means the column values never change, even - * across resets. Comparing to 'isConstant' above, this doesn't require any allocation of space. + * True if this column is missing from the file. This means the column values never change and are + * nulls, even across resets. This doesn't require any allocation of space. */ - protected boolean isAllNull; + protected boolean isMissing; /** * Default size of each array length value. This grows as necessary. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 4cc3fe61d22b4..08e545cb8c204 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -408,7 +408,7 @@ class ParquetFileFormat } override def supportDataType(dataType: DataType): Boolean = dataType match { - case _: AtomicType => true + case _: AtomicType | _: NullType => true case st: StructType => st.forall { f => supportDataType(f.dataType) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 25efd326f23a1..7ee5b4d224b34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -28,7 +28,7 @@ import org.apache.parquet.hadoop.api.{InitContext, ReadSupport} import org.apache.parquet.hadoop.api.ReadSupport.ReadContext import org.apache.parquet.io.api.RecordMaterializer import org.apache.parquet.schema._ -import org.apache.parquet.schema.LogicalTypeAnnotation.{ListLogicalTypeAnnotation, MapKeyValueTypeAnnotation, MapLogicalTypeAnnotation} +import org.apache.parquet.schema.LogicalTypeAnnotation.{ListLogicalTypeAnnotation, MapKeyValueTypeAnnotation, MapLogicalTypeAnnotation, UnknownLogicalTypeAnnotation} import org.apache.parquet.schema.Type.Repetition import org.apache.spark.internal.Logging @@ -562,6 +562,8 @@ object ParquetReadSupport extends Logging { } case primitiveType: PrimitiveType => val cost = primitiveType.getPrimitiveTypeName match { + case _ if primitiveType.getLogicalTypeAnnotation + .isInstanceOf[UnknownLogicalTypeAnnotation] => 0 // NullType is always preferred case PrimitiveType.PrimitiveTypeName.BOOLEAN => 1 case PrimitiveType.PrimitiveTypeName.INT32 => 4 case PrimitiveType.PrimitiveTypeName.INT64 => 8 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index f9d50bf28ea85..d708a19dd1ac7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -315,6 +315,13 @@ private[parquet] class ParquetRowConverter( } catalystType match { + case NullType + if parquetType.getLogicalTypeAnnotation.isInstanceOf[UnknownLogicalTypeAnnotation] => + val parentUpdater = updater + // A converter that throws upon any add... call, as we don't expect any value for NullType. + new PrimitiveConverter with HasParentContainerUpdater { + override def updater: ParentContainerUpdater = parentUpdater + } case LongType if isUnsignedIntTypeMatched(32) => new ParquetPrimitiveConverter(updater) { override def addInt(value: Int): Unit = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 947c021c1bd3a..76e24eb03f384 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -246,7 +246,9 @@ class ParquetToSparkSchemaConverter( DecimalType(precision, scale) } - val sparkType = sparkReadType.getOrElse(typeName match { + val isUnknownType = typeAnnotation.isInstanceOf[UnknownLogicalTypeAnnotation] + val nullTypeOpt = Option.when(isUnknownType)(NullType) + val sparkType = sparkReadType.orElse(nullTypeOpt).getOrElse(typeName match { case BOOLEAN => BooleanType case FLOAT => FloatType @@ -836,6 +838,10 @@ class SparkToParquetSchemaConverter( case udt: UserDefinedType[_] => convertField(field.copy(dataType = udt.sqlType), inShredded) + case NullType => // Selected primitive type here doesn't have significance. + Types.primitive(BOOLEAN, repetition).named(field.name) + .withLogicalTypeAnnotation(LogicalTypeAnnotation.unknownType()) + case _ => throw QueryCompilationErrors.cannotConvertDataTypeToParquetTypeError(field) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index 65a77322549f1..80a32711c50a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, Outpu import org.apache.spark.sql.execution.datasources.v2.V2ColumnUtils import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.internal.SQLConf.PARQUET_AGGREGATE_PUSHDOWN_ENABLED -import org.apache.spark.sql.types.{ArrayType, AtomicType, DataType, MapType, StructField, StructType, UserDefinedType, VariantType} +import org.apache.spark.sql.types.{ArrayType, AtomicType, DataType, MapType, NullType, StructField, StructType, UserDefinedType, VariantType} import org.apache.spark.util.ArrayImplicits._ object ParquetUtils extends Logging { @@ -209,6 +209,8 @@ object ParquetUtils extends Logging { def isBatchReadSupported(sqlConf: SQLConf, dt: DataType): Boolean = dt match { case _: AtomicType => true + case _: NullType => + sqlConf.parquetVectorizedReaderNullTypeEnabled case at: ArrayType => sqlConf.parquetVectorizedReaderNestedColumnEnabled && isBatchReadSupported(sqlConf, at.elementType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 2ab9fb64da43d..dcaf88fa8dfdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext import org.apache.parquet.io.api.{Binary, RecordConsumer} -import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} +import org.apache.spark.{SPARK_VERSION_SHORT, SparkException, SparkUnsupportedOperationException} import org.apache.spark.internal.Logging import org.apache.spark.sql.{SPARK_LEGACY_DATETIME_METADATA_KEY, SPARK_LEGACY_INT96_METADATA_KEY, SPARK_TIMEZONE_METADATA_KEY, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.catalyst.InternalRow @@ -192,6 +192,9 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { // schema. This affects how timestamp values are written. private def makeWriter(dataType: DataType, inShredded: Boolean): ValueWriter = { dataType match { + case NullType => // No values of NullType should ever be written, as all values are null. + (_: SpecializedGetters, _: Int) => throw SparkUnsupportedOperationException() + case BooleanType => (row: SpecializedGetters, ordinal: Int) => recordConsumer.addBoolean(row.getBoolean(ordinal)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 35bbc6c8a1f4b..95e86fe431198 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -565,10 +565,10 @@ class FileBasedDataSourceSuite extends QueryTest } } - test("SPARK-24204 error handling for unsupported Null data types - csv, parquet, orc") { + test("SPARK-24204 error handling for unsupported Null data types - csv, orc") { Seq(true, false).foreach { useV1 => val useV1List = if (useV1) { - "csv,orc,parquet" + "csv,orc" } else { "" } @@ -576,7 +576,7 @@ class FileBasedDataSourceSuite extends QueryTest withTempDir { dir => val tempDir = new File(dir, "files").getCanonicalPath - Seq("parquet", "csv", "orc").foreach { format => + Seq("csv", "orc").foreach { format => // write path checkError( exception = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index c237f5cc42fb3..3072657a0954f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -296,6 +296,31 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } + test("SPARK-54220: NullType") { + val data = (1 to 5) + .map(_ => Row(null, Row(null, null), Seq(null, null), Map(Row(null) -> null))).asJava + val dataSchema = new StructType() + .add("_1", NullType) + .add("_2", new StructType() + .add("_1", NullType) + .add("_2", NullType)) + .add("_3", ArrayType(NullType, containsNull = true)) + .add("_4", MapType(new StructType().add("_1", NullType), NullType, valueContainsNull = true)) + val expected = data.asScala.toArray + + withAllParquetWriters { + withTempPath { path => + val file = path.getCanonicalPath + spark.createDataFrame(data, dataSchema).write.parquet(file) + + withAllParquetReaders { + val df = spark.read.parquet(file) + checkAnswer(df, expected) + } + } + } + } + testStandardAndLegacyModes("map") { val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) checkParquetFile(data) @@ -928,6 +953,38 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } + test("SPARK-54220: vectorized reader: missing all struct fields, struct with NullType only") { + val data = Seq( + Tuple1((null, null)), + Tuple1((null, null)), + Tuple1(null) + ) + val readSchema = new StructType().add("_1", + new StructType() + .add("_3", IntegerType, nullable = true) + .add("_4", StringType, nullable = true), + nullable = true) + val expectedAnswer = Row(Row(null, null)) :: Row(Row(null, null)) :: Row(null) :: Nil + + withParquetFile(data) { file => + for (offheapEnabled <- Seq(true, false)) { + withSQLConf( + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "true", + SQLConf.LEGACY_PARQUET_RETURN_NULL_STRUCT_IF_ALL_FIELDS_MISSING.key -> "false", + SQLConf.COLUMN_VECTOR_OFFHEAP_ENABLED.key -> offheapEnabled.toString) { + withAllParquetReaders { + val df = spark.read.schema(readSchema).parquet(file) + val scanNode = df.queryExecution.executedPlan.collectLeaves().head + if (scanNode.supportsColumnar) { + VerifyNoAdditionalScanOutputExec(scanNode).execute().collect() + } + checkAnswer(df, expectedAnswer) + } + } + } + } + } + test("vectorized reader: missing some struct fields") { Seq(true, false).foreach { offheapEnabled => withSQLConf( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index d256752c287c9..56076175d60e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -3370,6 +3370,40 @@ class ParquetSchemaSuite extends ParquetSchemaTest { |} """).stripMargin, returnNullStructIfAllFieldsMissing = returnNullStructIfAllFieldsMissing) + + testSchemaClipping( + s"SPARK-54220: missing struct with NullType, " + + s"returnNullStructIfAllFieldsMissing=$returnNullStructIfAllFieldsMissing", + parquetSchema = + """message root { + | optional group _1 { + | optional int32 _1; + | optional boolean _2; + | optional group _3 { + | optional int64 _1 (UNKNOWN); + | } + | } + |} + """.stripMargin, + catalystSchema = new StructType() + .add("_1", new StructType() + .add("_101", IntegerType) + .add("_102", LongType)), + expectedSchema = + ("""message root { + | optional group _1 { + | optional int32 _101; + | optional int64 _102;""" + (if (!returnNullStructIfAllFieldsMissing) { + """ + | optional group _3 { + | optional int64 _1 (UNKNOWN); + | } + | }""" } else { "" }) + + """ + | } + |} + """).stripMargin, + returnNullStructIfAllFieldsMissing = returnNullStructIfAllFieldsMissing) } testSchemaClipping( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index caa4ca4581b4d..baafdc1ea50a3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2588,16 +2588,8 @@ class HiveDDLSuite test("SPARK-36241: support creating tables with void datatype") { // CTAS with void type withTable("t1", "t2", "t3") { - checkError( - exception = intercept[AnalysisException] { - sql("CREATE TABLE t1 USING PARQUET AS SELECT NULL AS null_col") - }, - condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", - parameters = Map( - "columnName" -> "`null_col`", - "columnType" -> "\"VOID\"", - "format" -> "Parquet") - ) + sql("CREATE TABLE t1 USING PARQUET AS SELECT NULL AS null_col") + checkAnswer(sql("SELECT * FROM t1"), Row(null)) checkError( exception = intercept[AnalysisException] { @@ -2615,15 +2607,8 @@ class HiveDDLSuite // Create table with void type withTable("t1", "t2", "t3", "t4") { - checkError( - exception = intercept[AnalysisException] { - sql("CREATE TABLE t1 (v VOID) USING PARQUET") - }, - condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", - parameters = Map( - "columnName" -> "`v`", - "columnType" -> "\"VOID\"", - "format" -> "Parquet")) + sql("CREATE TABLE t1 (v VOID) USING PARQUET") + checkAnswer(sql("SELECT * FROM t1"), Seq.empty) checkError( exception = intercept[AnalysisException] { From 23c0371259580ae49891e375cd53bb6609550f8f Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Fri, 7 Nov 2025 12:17:06 -0800 Subject: [PATCH 066/400] [SPARK-54229][PYTHON] Make PySparkLogger in UDFs store one log entry per log function call ### What changes were proposed in this pull request? Makes `PySparkLogger` in UDFs store one log entry per log function call. ### Why are the changes needed? Currently if `PySparkLogger` is used in UDFs, it will produce two entries per one log function call because it automatically adds a handler that writes to `sys.stderr`, which causes two entries. It doesn't need the additional handler if it's in the `capture_outputs` context.
example ```python >>> from pyspark.sql.functions import * >>> from pyspark.logger import PySparkLogger >>> >>> udf ... def pyspark_logger_test_udf(x): ... logger = PySparkLogger.getLogger("test") ... logger.warn(f"WARN level message: {x}", x=x) ... return str(x) ... >>> >>> spark.conf.set("spark.sql.pyspark.worker.logging.enabled", True) >>> >>> spark.range(1).select(pyspark_logger_test_udf("id")).show() ... ```
- before ```py >>> spark.table("system.session.python_worker_logs").orderBy("ts").show(truncate=False) +--------------------------+-------+----------------------------------------------------------------------------------------------------------------------------+----------------------------------------------+---------+------+ |ts |level |msg |context |exception|logger| +--------------------------+-------+----------------------------------------------------------------------------------------------------------------------------+----------------------------------------------+---------+------+ |2025-11-06 18:40:03.658127|WARNING|WARN level message: 0 |{func_name -> pyspark_logger_test_udf, x -> 0}|NULL |test | |2025-11-06 18:40:03.66424 |ERROR |{"ts": "2025-11-06 18:40:03.658", "level": "WARNING", "logger": "test", "msg": "WARN level message: 0", "context": {"x": 0}}|{func_name -> pyspark_logger_test_udf} |NULL |stderr| +--------------------------+-------+----------------------------------------------------------------------------------------------------------------------------+----------------------------------------------+---------+------+ ``` - after ```py >>> spark.table("system.session.python_worker_logs").orderBy("ts").show(truncate=False) +--------------------------+-------+---------------------+----------------------------------------------+---------+------+ |ts |level |msg |context |exception|logger| +--------------------------+-------+---------------------+----------------------------------------------+---------+------+ |2025-11-06 18:41:48.601256|WARNING|WARN level message: 0|{func_name -> pyspark_logger_test_udf, x -> 0}|NULL |test | +--------------------------+-------+---------------------+----------------------------------------------+---------+------+ ``` ### Does this PR introduce _any_ user-facing change? Yes, `PySparkLogger` in UDFs will store one log entry per one log function call. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52926 from ueshin/issues/SPARK-54229/pyspark_logger. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin (cherry picked from commit cc7f5124057fed783e311ec98322463cdeb8a72c) Signed-off-by: Takuya Ueshin --- python/pyspark/logger/logger.py | 12 +++++++ python/pyspark/sql/tests/test_udf.py | 30 ++++++++++++++++++ python/pyspark/sql/tests/test_udtf.py | 45 +++++++++++++++++++++++++++ 3 files changed, 87 insertions(+) diff --git a/python/pyspark/logger/logger.py b/python/pyspark/logger/logger.py index b60561f24c99c..72179b033bb36 100644 --- a/python/pyspark/logger/logger.py +++ b/python/pyspark/logger/logger.py @@ -140,7 +140,19 @@ class PySparkLogger(logging.Logger): """ def __init__(self, name: str = "PySparkLogger"): + from pyspark.logger.worker_io import JSONFormatterWithMarker + super().__init__(name, level=logging.WARN) + + root_logger = logging.getLogger() + if any( + isinstance(h, logging.StreamHandler) + and isinstance(h.formatter, JSONFormatterWithMarker) + for h in root_logger.handlers + ): + # Likely in the `capture_outputs` context, so don't add a handler + return + _handler = logging.StreamHandler() self.addHandler(_handler) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index d6bc8ad28b330..0ceb745c58607 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -48,6 +48,7 @@ VariantVal, ) from pyspark.errors import AnalysisException, PythonException, PySparkTypeError +from pyspark.logger import PySparkLogger from pyspark.testing.objects import ExamplePoint, ExamplePointUDT from pyspark.testing.sqlutils import ( ReusedSQLTestCase, @@ -1657,6 +1658,35 @@ def my_udf2(): ], ) + @unittest.skipIf(is_remote_only(), "Requires JVM access") + def test_udf_with_pyspark_logger(self): + @udf + def my_udf(x): + logger = PySparkLogger.getLogger("PySparkLogger") + logger.warning("PySparkLogger test", x=x) + return str(x) + + with self.sql_conf({"spark.sql.pyspark.worker.logging.enabled": "true"}): + assertDataFrameEqual( + self.spark.range(2).select(my_udf("id").alias("result")), + [Row(result=str(i)) for i in range(2)], + ) + + logs = self.spark.table("system.session.python_worker_logs") + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg="PySparkLogger test", + context={"func_name": my_udf.__name__, "x": str(i)}, + logger="PySparkLogger", + ) + for i in range(2) + ], + ) + class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase): @classmethod diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index b86a2624acd53..70623a5f62d44 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -65,6 +65,7 @@ StructType, VariantVal, ) +from pyspark.logger import PySparkLogger from pyspark.testing import assertDataFrameEqual, assertSchemaEqual from pyspark.testing.objects import ExamplePoint, ExamplePointUDT from pyspark.testing.sqlutils import ( @@ -3135,6 +3136,50 @@ def eval(self, x: int): ], ) + @unittest.skipIf(is_remote_only(), "Requires JVM access") + def test_udtf_analyze_with_pyspark_logger(self): + @udtf + class TestUDTFWithLogging: + @staticmethod + def analyze(x: AnalyzeArgument) -> AnalyzeResult: + logger = PySparkLogger.getLogger("PySparkLogger") + logger.warning(f"udtf analyze: {x.dataType.json()}", dt=x.dataType.json()) + return AnalyzeResult(StructType().add("a", IntegerType()).add("b", IntegerType())) + + def eval(self, x: int): + yield x * 2, x + 10 + + with self.sql_conf({"spark.sql.pyspark.worker.logging.enabled": "true"}): + assertDataFrameEqual( + self.spark.createDataFrame([(5,), (10,)], ["x"]).lateralJoin( + TestUDTFWithLogging(col("x").outer()) + ), + [Row(x=x, a=x * 2, b=x + 10) for x in [5, 10]], + ) + + logs = self.spark.table("system.session.python_worker_logs") + + assertDataFrameEqual( + logs.select( + "level", + "msg", + col("context.class_name").alias("context_class_name"), + col("context.func_name").alias("context_func_name"), + col("context.dt").alias("context_dt"), + "logger", + ).distinct(), + [ + Row( + level="WARNING", + msg='udtf analyze: "long"', + context_class_name="TestUDTFWithLogging", + context_func_name="analyze", + context_dt='"long"', + logger="PySparkLogger", + ) + ], + ) + class UDTFTests(BaseUDTFTestsMixin, ReusedSQLTestCase): @classmethod From 2ca18e4851266e56a73785a9a7261a0b6dc89af4 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 7 Nov 2025 13:03:12 -0800 Subject: [PATCH 067/400] [SPARK-54231][SDP] Fill gaps in SDP docs ### What changes were proposed in this pull request? Fill some gaps in the SDP docs: - Document storage field in pipeline spec - Update documentation libraries field to reflect new name - Provide installation instructions ### Why are the changes needed? To make the docs more accurate. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #52929 from sryza/sdp-docs-install-instructions. Authored-by: Sandy Ryza Signed-off-by: Wenchen Fan (cherry picked from commit bab98afd490b0aa553f7b5d9f75d65d204b66db0) Signed-off-by: Wenchen Fan --- ...declarative-pipelines-programming-guide.md | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/docs/declarative-pipelines-programming-guide.md b/docs/declarative-pipelines-programming-guide.md index 3e33153e3d252..3932d472cf659 100644 --- a/docs/declarative-pipelines-programming-guide.md +++ b/docs/declarative-pipelines-programming-guide.md @@ -35,6 +35,16 @@ The key advantage of SDP is its declarative approach - you define what tables sh ![Dataflow Graph](img/declarative-pipelines-dataflow-graph.png) +### Quick install + +A quick way to install SDP is with pip: + +``` +pip install pyspark[pipelines] +``` + +See the [downloads page](//spark.apache.org/downloads.html) for more installation options. + ## Key Concepts ### Flows @@ -67,8 +77,9 @@ A pipeline is the primary unit of development and execution in SDP. A pipeline c A pipeline project is a set of source files that contain code that define the datasets and flows that make up a pipeline. These source files can be `.py` or `.sql` files. A YAML-formatted pipeline spec file contains the top-level configuration for the pipeline project. It supports the following fields: -- **definitions** (Required) - Paths where definition files can be found. -- **database** (Optional) - The default target database for pipeline outputs. +- **libraries** (Required) - Paths where source files can be found. +- **storage** (Required) – A directory where checkpoints can be stored for streams within the pipeline. +- **database** (Optional) - The default target database for pipeline outputs. **schema** can alternatively be used as an alias. - **catalog** (Optional) - The default target catalog for pipeline outputs. - **configuration** (Optional) - Map of Spark configuration properties. @@ -76,11 +87,9 @@ An example pipeline spec file: ```yaml name: my_pipeline -definitions: - - glob: - include: transformations/**/*.py +libraries: - glob: - include: transformations/**/*.sql + include: transformations/** catalog: my_catalog database: my_db configuration: From 7b7bca875ca9c53169eac65645ca8299a83d14a1 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Fri, 7 Nov 2025 13:06:28 -0800 Subject: [PATCH 068/400] [SPARK-54232][GEO][CONNECT] Enable Arrow serialization for Geography and Geometry types ### What changes were proposed in this pull request? Introduce Arrow serialization/deserialization for `Geography` and `Geometry`. ### Why are the changes needed? Enable geospatial result set serialization in Arrow format for Spark Connect and Thrift Server. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit tests: - `GeographyConnectDataFrameSuite` - `GeometryConnectDataFrameSuite` - `ArrowEncoderSuite` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52930 from uros-db/geo-arrow. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit f9191248900c5933a2934e938931f4fdc8b76f50) Signed-off-by: Wenchen Fan --- .../sql/GeographyConnectDataFrameSuite.scala | 106 +++++++++++++++++ .../sql/GeometryConnectDataFrameSuite.scala | 112 ++++++++++++++++++ .../client/arrow/ArrowEncoderSuite.scala | 98 ++++++++++++++- .../client/arrow/ArrowDeserializer.scala | 8 ++ .../client/arrow/ArrowEncoderUtils.scala | 16 +++ .../client/arrow/ArrowSerializer.scala | 14 ++- .../client/arrow/GeospatialArrowSerDe.scala | 101 ++++++++++++++++ .../common/DataTypeProtoConverter.scala | 35 ++++++ 8 files changed, 487 insertions(+), 3 deletions(-) create mode 100644 sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeographyConnectDataFrameSuite.scala create mode 100644 sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeometryConnectDataFrameSuite.scala create mode 100644 sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/GeospatialArrowSerDe.scala diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeographyConnectDataFrameSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeographyConnectDataFrameSuite.scala new file mode 100644 index 0000000000000..2016a84ac5a35 --- /dev/null +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeographyConnectDataFrameSuite.scala @@ -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.spark.sql + +import scala.collection.immutable.Seq + +import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} +import org.apache.spark.sql.connect.test.{QueryTest, RemoteSparkSession} +import org.apache.spark.sql.types._ + +class GeographyConnectDataFrameSuite extends QueryTest with RemoteSparkSession { + + private val point1: Array[Byte] = "010100000000000000000031400000000000001C40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + private val point2: Array[Byte] = "010100000000000000000035400000000000001E40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + + test("decode geography value: SRID schema does not match input SRID data schema") { + val geography = Geography.fromWKB(point1, 0) + + val seq = Seq((geography, 1)) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(seq).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326")) + + import testImplicits._ + checkError( + exception = intercept[SparkRuntimeException] { + Seq(geography).toDF().collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOGRAPHY", "valueSrid" -> "0", "typeSrid" -> "4326")) + } + + test("decode geography value: mixed SRID schema is provided") { + val schema = StructType(Seq(StructField("col1", GeographyType("ANY"), nullable = false))) + val expectedResult = + Seq(Row(Geography.fromWKB(point1, 4326)), Row(Geography.fromWKB(point2, 4326))) + + val javaList = java.util.Arrays + .asList(Row(Geography.fromWKB(point1, 4326)), Row(Geography.fromWKB(point2, 4326))) + val resultJavaListDF = spark.createDataFrame(javaList, schema) + checkAnswer(resultJavaListDF, expectedResult) + + // Test that unsupported SRID with mixed schema will throw an error. + val invalidData = + java.util.Arrays + .asList(Row(Geography.fromWKB(point1, 1)), Row(Geography.fromWKB(point2, 4326))) + checkError( + exception = intercept[SparkIllegalArgumentException] { + spark.createDataFrame(invalidData, schema).collect() + }, + condition = "ST_INVALID_SRID_VALUE", + parameters = Map("srid" -> "1")) + } + + test("createDataFrame APIs with Geography.fromWKB") { + val geography1 = Geography.fromWKB(point1, 4326) + val geography2 = Geography.fromWKB(point2) + + val seq = Seq((geography1, 1), (geography2, 2), (null, 3)) + val dfFromSeq = spark.createDataFrame(seq) + checkAnswer(dfFromSeq, Seq(Row(geography1, 1), Row(geography2, 2), Row(null, 3))) + + val schema = StructType(Seq(StructField("geography", GeographyType(4326), nullable = true))) + + val javaList = java.util.Arrays.asList(Row(geography1), Row(geography2), Row(null)) + val dfFromJavaList = spark.createDataFrame(javaList, schema) + checkAnswer(dfFromJavaList, Seq(Row(geography1), Row(geography2), Row(null))) + + import testImplicits._ + val implicitDf = Seq(geography1, geography2, null).toDF() + checkAnswer(implicitDf, Seq(Row(geography1), Row(geography2), Row(null))) + } + + test("encode geography type") { + // POINT (17 7) + val wkb = "010100000000000000000031400000000000001C40" + val df = spark.sql(s"SELECT ST_GeogFromWKB(X'$wkb')") + val point = wkb.grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val expectedGeog = Geography.fromWKB(point, 4326) + checkAnswer(df, Seq(Row(expectedGeog))) + } +} diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeometryConnectDataFrameSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeometryConnectDataFrameSuite.scala new file mode 100644 index 0000000000000..1450ac54184bd --- /dev/null +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/GeometryConnectDataFrameSuite.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.collection.immutable.Seq + +import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} +import org.apache.spark.sql.connect.test.{QueryTest, RemoteSparkSession} +import org.apache.spark.sql.types._ + +class GeometryConnectDataFrameSuite extends QueryTest with RemoteSparkSession { + + private val point1: Array[Byte] = "010100000000000000000031400000000000001C40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + private val point2: Array[Byte] = "010100000000000000000035400000000000001E40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + + test("decode geometry value: SRID schema does not match input SRID data schema") { + val geometry = Geometry.fromWKB(point1, 4326) + + val seq = Seq((geometry, 1)) + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataFrame(seq).collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "0")) + + import testImplicits._ + checkError( + exception = intercept[SparkRuntimeException] { + Seq(geometry).toDF().collect() + }, + condition = "GEO_ENCODER_SRID_MISMATCH_ERROR", + parameters = Map("type" -> "GEOMETRY", "valueSrid" -> "4326", "typeSrid" -> "0")) + } + + test("decode geometry value: mixed SRID schema is provided") { + val schema = StructType(Seq(StructField("col1", GeometryType("ANY"), nullable = false))) + val expectedResult = + Seq(Row(Geometry.fromWKB(point1, 0)), Row(Geometry.fromWKB(point2, 4326))) + + val javaList = java.util.Arrays + .asList(Row(Geometry.fromWKB(point1, 0)), Row(Geometry.fromWKB(point2, 4326))) + val resultJavaListDF = spark.createDataFrame(javaList, schema) + checkAnswer(resultJavaListDF, expectedResult) + + // Test that unsupported SRID with mixed schema will throw an error. + val invalidData = + java.util.Arrays + .asList(Row(Geometry.fromWKB(point1, 1)), Row(Geometry.fromWKB(point2, 4326))) + checkError( + exception = intercept[SparkIllegalArgumentException] { + spark.createDataFrame(invalidData, schema).collect() + }, + condition = "ST_INVALID_SRID_VALUE", + parameters = Map("srid" -> "1")) + } + + test("createDataFrame APIs with Geometry.fromWKB") { + val geometry1 = Geometry.fromWKB(point1, 0) + val geometry2 = Geometry.fromWKB(point2, 0) + + // 1. Test createDataFrame with Seq of Geometry objects + val seq = Seq((geometry1, 1), (geometry2, 2), (null, 3)) + val dfFromSeq = spark.createDataFrame(seq) + checkAnswer(dfFromSeq, Seq(Row(geometry1, 1), Row(geometry2, 2), Row(null, 3))) + + // 2. Test createDataFrame with RDD of Rows and StructType schema + val geometry3 = Geometry.fromWKB(point1, 4326) + val geometry4 = Geometry.fromWKB(point2, 4326) + val schema = StructType(Seq(StructField("geometry", GeometryType(4326), nullable = true))) + + // 3. Test createDataFrame with Java List of Rows and StructType schema + val javaList = java.util.Arrays.asList(Row(geometry3), Row(geometry4), Row(null)) + val dfFromJavaList = spark.createDataFrame(javaList, schema) + checkAnswer(dfFromJavaList, Seq(Row(geometry3), Row(geometry4), Row(null))) + + // 4. Implicit conversion from Seq to DF + import testImplicits._ + val implicitDf = Seq(geometry1, geometry2, null).toDF() + checkAnswer(implicitDf, Seq(Row(geometry1), Row(geometry2), Row(null))) + } + + test("encode geometry type") { + // POINT (17 7) + val wkb = "010100000000000000000031400000000000001C40" + val df = spark.sql(s"SELECT ST_GeomFromWKB(X'$wkb')") + val point = wkb.grouped(2).map(Integer.parseInt(_, 16).toByte).toArray + val expectedGeom = Geometry.fromWKB(point, 0) + checkAnswer(df, Seq(Row(expectedGeom))) + } +} diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala index bc840df5c3fac..d24369ff5fc7c 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.catalyst.util.SparkIntervalUtils._ import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.connect.client.arrow.FooEnum.FooEnum import org.apache.spark.sql.connect.test.ConnectFunSuite -import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType} +import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, Geography, Geometry, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType} import org.apache.spark.unsafe.types.VariantVal import org.apache.spark.util.{MaybeNull, SparkStringUtils} @@ -263,6 +263,102 @@ class ArrowEncoderSuite extends ConnectFunSuite with BeforeAndAfterAll { assert(inspector.numBatches == 1) } + test("geography round trip") { + val point1 = "010100000000000000000031400000000000001C40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + val point2 = "010100000000000000000035400000000000001E40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + + val geographyEncoder = toRowEncoder(new StructType().add("g", "geography(4326)")) + roundTripAndCheckIdentical(geographyEncoder) { () => + val maybeNull = MaybeNull(7) + Iterator.tabulate(101)(i => Row(maybeNull(Geography.fromWKB(point1, 4326)))) + } + + val nestedGeographyEncoder = toRowEncoder( + new StructType() + .add( + "s", + new StructType() + .add("i1", "int") + .add("g0", "geography(4326)") + .add("i2", "int") + .add("g4326", "geography(4326)")) + .add("a", "array") + .add("m", "map")) + + roundTripAndCheckIdentical(nestedGeographyEncoder) { () => + val maybeNull5 = MaybeNull(5) + val maybeNull7 = MaybeNull(7) + val maybeNull11 = MaybeNull(11) + val maybeNull13 = MaybeNull(13) + val maybeNull17 = MaybeNull(17) + Iterator + .tabulate(100)(i => + Row( + maybeNull5( + Row( + i, + maybeNull7(Geography.fromWKB(point1)), + i + 1, + maybeNull11(Geography.fromWKB(point2, 4326)))), + maybeNull7((0 until 10).map(j => Geography.fromWKB(point2, 0))), + maybeNull13(Map((i.toString, maybeNull17(Geography.fromWKB(point1, 4326))))))) + } + } + + test("geometry round trip") { + val point1 = "010100000000000000000031400000000000001C40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + val point2 = "010100000000000000000035400000000000001E40" + .grouped(2) + .map(Integer.parseInt(_, 16).toByte) + .toArray + + val geometryEncoder = toRowEncoder(new StructType().add("g", "geometry(0)")) + roundTripAndCheckIdentical(geometryEncoder) { () => + val maybeNull = MaybeNull(7) + Iterator.tabulate(101)(i => Row(maybeNull(Geometry.fromWKB(point1, 0)))) + } + + val nestedGeometryEncoder = toRowEncoder( + new StructType() + .add( + "s", + new StructType() + .add("i1", "int") + .add("g0", "geometry(0)") + .add("i2", "int") + .add("g4326", "geometry(4326)")) + .add("a", "array") + .add("m", "map")) + + roundTripAndCheckIdentical(nestedGeometryEncoder) { () => + val maybeNull5 = MaybeNull(5) + val maybeNull7 = MaybeNull(7) + val maybeNull11 = MaybeNull(11) + val maybeNull13 = MaybeNull(13) + val maybeNull17 = MaybeNull(17) + Iterator + .tabulate(100)(i => + Row( + maybeNull5( + Row( + i, + maybeNull7(Geometry.fromWKB(point1, 0)), + i + 1, + maybeNull11(Geometry.fromWKB(point2, 4326)))), + maybeNull7((0 until 10).map(j => Geometry.fromWKB(point2, 0))), + maybeNull13(Map((i.toString, maybeNull17(Geometry.fromWKB(point1, 4326))))))) + } + } + test("variant round trip") { val variantEncoder = toRowEncoder(new StructType().add("v", "variant")) roundTripAndCheckIdentical(variantEncoder) { () => diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index 7597a0ceeb8cd..8d5811dda8f35 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -341,6 +341,14 @@ object ArrowDeserializers { } } + case (_: GeometryEncoder, StructVectors(struct, vectors)) => + val gdser = new GeometryArrowSerDe + gdser.createDeserializer(struct, vectors, timeZoneId) + + case (_: GeographyEncoder, StructVectors(struct, vectors)) => + val gdser = new GeographyArrowSerDe + gdser.createDeserializer(struct, vectors, timeZoneId) + case (VariantEncoder, StructVectors(struct, vectors)) => assert(vectors.exists(_.getName == "value")) assert( diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala index 5b1539e39f4f4..2430c2bbc86fc 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala @@ -41,6 +41,22 @@ private[arrow] object ArrowEncoderUtils { def unsupportedCollectionType(cls: Class[_]): Nothing = { throw new RuntimeException(s"Unsupported collection type: $cls") } + + def assertMetadataPresent( + vectors: Seq[FieldVector], + expectedVectors: Seq[String], + expectedMetadata: Seq[(String, String)]): Unit = { + expectedVectors.foreach { vectorName => + assert(vectors.exists(_.getName == vectorName)) + } + + expectedVectors.zip(expectedMetadata).foreach { case (vectorName, (key, value)) => + assert( + vectors.exists(field => + field.getName == vectorName && field.getField.getMetadata + .containsKey(key) && field.getField.getMetadata.get(key) == value)) + } + } } private[arrow] object StructVectors { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala index 4acb11f014d19..73c9a991ab6a7 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala @@ -487,6 +487,14 @@ object ArrowSerializer { extractor = (v: Any) => v.asInstanceOf[VariantVal].getMetadata, serializerFor(BinaryEncoder, struct.getChild("metadata"))))) + case (_: GeographyEncoder, StructVectors(struct, vectors)) => + val gser = new GeographyArrowSerDe + gser.createSerializer(struct, vectors) + + case (_: GeometryEncoder, StructVectors(struct, vectors)) => + val gser = new GeometryArrowSerDe + gser.createSerializer(struct, vectors) + case (JavaBeanEncoder(tag, fields), StructVectors(struct, vectors)) => structSerializerFor(fields, struct, vectors) { (field, _) => val getter = methodLookup.findVirtual( @@ -585,12 +593,14 @@ object ArrowSerializer { } } - private class StructFieldSerializer(val extractor: Any => Any, val serializer: Serializer) { + private[arrow] class StructFieldSerializer( + val extractor: Any => Any, + val serializer: Serializer) { def write(index: Int, value: Any): Unit = serializer.write(index, extractor(value)) def writeNull(index: Int): Unit = serializer.write(index, null) } - private class StructSerializer( + private[arrow] class StructSerializer( struct: StructVector, fieldSerializers: Seq[StructFieldSerializer]) extends Serializer { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/GeospatialArrowSerDe.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/GeospatialArrowSerDe.scala new file mode 100644 index 0000000000000..443523ef02cdc --- /dev/null +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/GeospatialArrowSerDe.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.client.arrow + +import org.apache.arrow.vector.FieldVector +import org.apache.arrow.vector.complex.StructVector + +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, PrimitiveIntEncoder} +import org.apache.spark.sql.errors.CompilationErrors +import org.apache.spark.sql.types.{Geography, Geometry} + +abstract class GeospatialArrowSerDe[T](typeName: String) { + + def createDeserializer( + struct: StructVector, + vectors: Seq[FieldVector], + timeZoneId: String): ArrowDeserializers.StructFieldSerializer[T] = { + assertMetadataPresent(vectors) + val wkbDecoder = ArrowDeserializers.deserializerFor( + BinaryEncoder, + vectors + .find(_.getName == "wkb") + .getOrElse(throw CompilationErrors.columnNotFoundError("wkb")), + timeZoneId) + val sridDecoder = ArrowDeserializers.deserializerFor( + PrimitiveIntEncoder, + vectors + .find(_.getName == "srid") + .getOrElse(throw CompilationErrors.columnNotFoundError("srid")), + timeZoneId) + new ArrowDeserializers.StructFieldSerializer[T](struct) { + override def value(i: Int): T = createInstance(wkbDecoder.get(i), sridDecoder.get(i)) + } + } + + def createSerializer( + struct: StructVector, + vectors: Seq[FieldVector]): ArrowSerializer.StructSerializer = { + assertMetadataPresent(vectors) + new ArrowSerializer.StructSerializer( + struct, + Seq( + new ArrowSerializer.StructFieldSerializer( + extractor = (v: Any) => extractSrid(v), + ArrowSerializer.serializerFor(PrimitiveIntEncoder, struct.getChild("srid"))), + new ArrowSerializer.StructFieldSerializer( + extractor = (v: Any) => extractBytes(v), + ArrowSerializer.serializerFor(BinaryEncoder, struct.getChild("wkb"))))) + } + + private def assertMetadataPresent(vectors: Seq[FieldVector]): Unit = { + assert(vectors.exists(_.getName == "srid")) + assert( + vectors.exists(field => + field.getName == "wkb" && field.getField.getMetadata + .containsKey(typeName) && field.getField.getMetadata.get(typeName) == "true")) + } + + protected def createInstance(wkb: Any, srid: Any): T + protected def extractSrid(value: Any): Int + protected def extractBytes(value: Any): Array[Byte] +} + +// Geography-specific implementation +class GeographyArrowSerDe extends GeospatialArrowSerDe[Geography]("geography") { + override protected def createInstance(wkb: Any, srid: Any): Geography = + Geography.fromWKB(wkb.asInstanceOf[Array[Byte]], srid.asInstanceOf[Int]) + + override protected def extractSrid(value: Any): Int = + value.asInstanceOf[Geography].getSrid + + override protected def extractBytes(value: Any): Array[Byte] = + value.asInstanceOf[Geography].getBytes +} + +// Geometry-specific implementation +class GeometryArrowSerDe extends GeospatialArrowSerDe[Geometry]("geometry") { + override protected def createInstance(wkb: Any, srid: Any): Geometry = + Geometry.fromWKB(wkb.asInstanceOf[Array[Byte]], srid.asInstanceOf[Int]) + + override protected def extractSrid(value: Any): Int = + value.asInstanceOf[Geometry].getSrid + + override protected def extractBytes(value: Any): Array[Byte] = + value.asInstanceOf[Geometry].getBytes +} diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala index 419cc8e082af2..ac69f084c307b 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala @@ -71,6 +71,21 @@ object DataTypeProtoConverter { case proto.DataType.KindCase.MAP => toCatalystMapType(t.getMap) case proto.DataType.KindCase.VARIANT => VariantType + case proto.DataType.KindCase.GEOMETRY => + val srid = t.getGeometry.getSrid + if (srid == GeometryType.MIXED_SRID) { + GeometryType("ANY") + } else { + GeometryType(srid) + } + case proto.DataType.KindCase.GEOGRAPHY => + val srid = t.getGeography.getSrid + if (srid == GeographyType.MIXED_SRID) { + GeographyType("ANY") + } else { + GeographyType(srid) + } + case proto.DataType.KindCase.UDT => toCatalystUDT(t.getUdt) case _ => @@ -307,6 +322,26 @@ object DataTypeProtoConverter { .build()) .build() + case g: GeographyType => + proto.DataType + .newBuilder() + .setGeography( + proto.DataType.Geography + .newBuilder() + .setSrid(g.srid) + .build()) + .build() + + case g: GeometryType => + proto.DataType + .newBuilder() + .setGeometry( + proto.DataType.Geometry + .newBuilder() + .setSrid(g.srid) + .build()) + .build() + case VariantType => ProtoDataTypes.VariantType case pyudt: PythonUserDefinedType => From 706985832285a71ed253a7b3ce9c677624b0787a Mon Sep 17 00:00:00 2001 From: Jerry Peng Date: Fri, 7 Nov 2025 13:08:34 -0800 Subject: [PATCH 069/400] [SPARK-53823][SS] Implement allow list for real time mode ### What changes were proposed in this pull request? Add an allowlist and some guardrails to help users understand what is supported and what is not supported in Real-time Mode. This should improve the user experience of real-time mode and minimize the chance of a new user use it in a way that is unexpected or untested which may produce undesirable outcomes. ### Why are the changes needed? To help guide users on what is currently supported and what is not. ### Does this PR introduce _any_ user-facing change? Yes, running a query with currently unsupported features in RTM will throw an exception. ### How was this patch tested? Several tests are added ### Was this patch authored or co-authored using generative AI tooling? No Closes #52891 from jerrypeng/SPARK-53823. Authored-by: Jerry Peng Signed-off-by: Dongjoon Hyun (cherry picked from commit 178f0f428020dd1795313afe1c6dd0c125e1b70d) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 15 ++ .../UnsupportedOperationChecker.scala | 19 ++ .../streaming/WriteToStreamStatement.scala | 8 +- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../analysis/UnsupportedOperationsSuite.scala | 46 +++++ .../spark/sql/classic/DataStreamWriter.scala | 9 + .../sql/classic/StreamingQueryManager.scala | 2 +- .../runtime/MicroBatchExecution.scala | 15 +- .../runtime/RealTimeModeAllowlist.scala | 148 ++++++++++++++++ .../runtime/ResolveWriteToStream.scala | 8 +- .../StreamRealTimeModeAllowlistSuite.scala | 163 ++++++++++++++++++ 11 files changed, 433 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RealTimeModeAllowlist.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeAllowlistSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 55f99208e0a15..49a217652791d 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5629,6 +5629,21 @@ "message" : [ "The input stream is not supported in Real-time Mode." ] + }, + "OPERATOR_OR_SINK_NOT_IN_ALLOWLIST" : { + "message" : [ + "The (s): not in the allowlist for Real-Time Mode. To bypass this check, set spark.sql.streaming.realTimeMode.allowlistCheck to false. By changing this, you agree to run the query at your own risk." + ] + }, + "OUTPUT_MODE_NOT_SUPPORTED" : { + "message" : [ + "The output mode is not supported. To work around this limitation, set the output mode to Update. In the future, may be supported." + ] + }, + "SINK_NOT_SUPPORTED" : { + "message" : [ + "The sink is currently not supported. See the Real-Time Mode User Guide for a list of supported sinks." + ] } }, "sqlState" : "0A000" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index fd4e081c91b52..d658d83f066f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -584,6 +584,25 @@ object UnsupportedOperationChecker extends Logging { } } + // Verifies that a query using real-time mode is valid. It is meant to be used in addition to + // the checkForStreaming method: for this reason, we call this method check *additional* + // real-time mode constraints. + // + // It should be called during resolution of the WriteToStreamStatement if and only if + // the query is using the real-time trigger. + def checkAdditionalRealTimeModeConstraints(plan: LogicalPlan, outputMode: OutputMode): Unit = { + if (outputMode != InternalOutputModes.Update) { + throwRealTimeError("OUTPUT_MODE_NOT_SUPPORTED", Map("outputMode" -> outputMode.toString)) + } + } + + private def throwRealTimeError(subClass: String, args: Map[String, String]): Unit = { + throw new AnalysisException( + errorClass = s"STREAMING_REAL_TIME_MODE.$subClass", + messageParameters = args + ) + } + private def throwErrorIf( condition: Boolean, msg: String)(implicit operator: LogicalPlan): Unit = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/WriteToStreamStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/WriteToStreamStatement.scala index a6204b317d249..7015d0dd3b2cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/WriteToStreamStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/WriteToStreamStatement.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} -import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.streaming.{OutputMode, Trigger} /** * A statement for Stream writing. It contains all neccessary param and will be resolved in the @@ -39,7 +39,9 @@ import org.apache.spark.sql.streaming.OutputMode * @param sink Sink to write the streaming outputs. * @param outputMode Output mode for the sink. * @param hadoopConf The Hadoop Configuration to get a FileSystem instance - * @param isContinuousTrigger Whether the statement is triggered by a continuous query or not. + * @param trigger The trigger being used for this streaming query. It is not used to create the + * resolved [[WriteToStream]] node; rather, it is only used while checking the plan + * for unsupported operations, which happens during resolution. * @param inputQuery The analyzed query plan from the streaming DataFrame. * @param catalogAndIdent Catalog and identifier for the sink, set when it is a V2 catalog table */ @@ -51,7 +53,7 @@ case class WriteToStreamStatement( sink: Table, outputMode: OutputMode, hadoopConf: Configuration, - isContinuousTrigger: Boolean, + trigger: Trigger, inputQuery: LogicalPlan, catalogAndIdent: Option[(TableCatalog, Identifier)] = None, catalogTable: Option[CatalogTable] = None) extends UnaryNode { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 366c5700e06f9..8f27c9097d3d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3039,6 +3039,13 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(5000) + val STREAMING_REAL_TIME_MODE_ALLOWLIST_CHECK = buildConf( + "spark.sql.streaming.realTimeMode.allowlistCheck") + .doc("Whether to check all operators, sinks used in real-time mode are in the allowlist.") + .version("4.1.0") + .booleanConf + .createWithDefault(true) + val VARIABLE_SUBSTITUTE_ENABLED = buildConf("spark.sql.variable.substitute") .doc("This enables substitution using syntax like `${var}`, `${system:var}`, " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 6ee19bab5180a..425df0856a58a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -853,6 +853,27 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { Deduplicate(Seq(attribute), streamRelation)), outputMode = Append) } + /* + ======================================================================================= + REAL-TIME STREAMING + ======================================================================================= + */ + + { + assertNotSupportedForRealTime( + "real-time without operators - append mode", + streamRelation, + Append, + "STREAMING_REAL_TIME_MODE.OUTPUT_MODE_NOT_SUPPORTED" + ) + + assertSupportedForRealTime( + "real-time with stream-batch join - update mode", + streamRelation.join(batchRelation, joinType = Inner), + Update + ) + } + /* ======================================================================================= TESTING FUNCTIONS @@ -1017,6 +1038,31 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { } } + /** Assert that the logical plan is supported for real-time mode */ + def assertSupportedForRealTime(name: String, plan: LogicalPlan, outputMode: OutputMode): Unit = { + test(s"real-time trigger - $name: supported") { + UnsupportedOperationChecker.checkAdditionalRealTimeModeConstraints(plan, outputMode) + } + } + + /** + * Assert that the logical plan is not supported inside a streaming plan with the + * real-time trigger. + */ + def assertNotSupportedForRealTime( + name: String, + plan: LogicalPlan, + outputMode: OutputMode, + condition: String): Unit = { + testError( + s"real-time trigger - $name: not supported", + Seq("Streaming real-time mode"), + condition + ) { + UnsupportedOperationChecker.checkAdditionalRealTimeModeConstraints(plan, outputMode) + } + } + /** * Assert that the logical plan is not supported inside a streaming plan. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala index 471c5feadaabc..38483395ec8c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala @@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDat import org.apache.spark.sql.execution.datasources.v2.python.PythonDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, Trigger} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ @@ -299,6 +300,14 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) extends streaming.D recoverFromCheckpoint: Boolean = true, catalogAndIdent: Option[(TableCatalog, Identifier)] = None, catalogTable: Option[CatalogTable] = None): StreamingQuery = { + if (trigger.isInstanceOf[RealTimeTrigger]) { + RealTimeModeAllowlist.checkAllowedSink( + sink, + ds.sparkSession.sessionState.conf.getConf( + SQLConf.STREAMING_REAL_TIME_MODE_ALLOWLIST_CHECK) + ) + } + val useTempCheckpointLocation = DataStreamWriter.SOURCES_ALLOW_ONE_TIME_QUERY.contains(source) ds.sparkSession.sessionState.streamingQueryManager.startQuery( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala index bef09703025ef..72ae3b21d662a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala @@ -213,7 +213,7 @@ class StreamingQueryManager private[sql] ( sink, outputMode, df.sparkSession.sessionState.newHadoopConf(), - trigger.isInstanceOf[ContinuousTrigger], + trigger, analyzedPlan, catalogAndIdent, catalogTable) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala index d4f9dc8cea93a..cf2fca3d3cd8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, RealTimeStreamScanExec, StreamingDataSourceV2Relation, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} -import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, Offset, OneTimeTrigger, ProcessingTimeTrigger, RealTimeTrigger, Sink, Source, StreamingQueryPlanTraverseHelper} +import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, Offset, OneTimeTrigger, ProcessingTimeTrigger, RealTimeModeAllowlist, RealTimeTrigger, Sink, Source, StreamingQueryPlanTraverseHelper} import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitMetadata, OffsetSeq, OffsetSeqMetadata} import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOpStateStoreCheckpointInfo, StateStoreWriter} import org.apache.spark.sql.execution.streaming.runtime.AcceptsLatestSeenOffsetHandler @@ -436,7 +436,10 @@ class MicroBatchExecution( } } - if (containsStatefulOperator(analyzedPlan)) { + if (trigger.isInstanceOf[RealTimeTrigger]) { + logWarning(log"Disabling AQE since AQE is not supported for Real-time Mode.") + sparkSessionToRunBatches.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } else if (containsStatefulOperator(analyzedPlan)) { // SPARK-53941: We disable AQE for stateful workloads as of now. logWarning(log"Disabling AQE since AQE is not supported in stateful workloads.") sparkSessionToRunBatches.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") @@ -1042,6 +1045,14 @@ class MicroBatchExecution( markMicroBatchExecutionStart(execCtx) + if (trigger.isInstanceOf[RealTimeTrigger]) { + RealTimeModeAllowlist.checkAllowedPhysicalOperator( + execCtx.executionPlan.executedPlan, + sparkSession.sessionState.conf.getConf( + SQLConf.STREAMING_REAL_TIME_MODE_ALLOWLIST_CHECK) + ) + } + if (execCtx.previousContext.isEmpty) { purgeStatefulMetadataAsync(execCtx.executionPlan.executedPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RealTimeModeAllowlist.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RealTimeModeAllowlist.scala new file mode 100644 index 0000000000000..443c7fa1a1cf6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RealTimeModeAllowlist.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.internal.{Logging, LogKeys, MessageWithContext} +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.RealTimeStreamScanExec +import org.apache.spark.sql.execution.streaming.operators.stateful._ + +object RealTimeModeAllowlist extends Logging { + private val allowedSinks = Set( + "org.apache.spark.sql.execution.streaming.ConsoleTable$", + "org.apache.spark.sql.execution.streaming.sources.ContinuousMemorySink", + "org.apache.spark.sql.execution.streaming.sources.ForeachWriterTable", + "org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable" + ) + + private val allowedOperators = Set( + "org.apache.spark.sql.execution.AppendColumnsExec", + "org.apache.spark.sql.execution.CollectMetricsExec", + "org.apache.spark.sql.execution.ColumnarToRowExec", + "org.apache.spark.sql.execution.DeserializeToObjectExec", + "org.apache.spark.sql.execution.ExpandExec", + "org.apache.spark.sql.execution.FileSourceScanExec", + "org.apache.spark.sql.execution.FilterExec", + "org.apache.spark.sql.execution.GenerateExec", + "org.apache.spark.sql.execution.InputAdapter", + "org.apache.spark.sql.execution.LocalTableScanExec", + "org.apache.spark.sql.execution.MapElementsExec", + "org.apache.spark.sql.execution.MapPartitionsExec", + "org.apache.spark.sql.execution.PlanLater", + "org.apache.spark.sql.execution.ProjectExec", + "org.apache.spark.sql.execution.RangeExec", + "org.apache.spark.sql.execution.SerializeFromObjectExec", + "org.apache.spark.sql.execution.UnionExec", + "org.apache.spark.sql.execution.WholeStageCodegenExec", + "org.apache.spark.sql.execution.datasources.v2.RealTimeStreamScanExec", + "org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec", + "org.apache.spark.sql.execution.exchange.BroadcastExchangeExec", + "org.apache.spark.sql.execution.exchange.ReusedExchangeExec", + "org.apache.spark.sql.execution.joins.BroadcastHashJoinExec", + classOf[EventTimeWatermarkExec].getName + ) + + private def classNamesString(classNames: Seq[String]): MessageWithContext = { + val sortedClassNames = classNames.sorted + var message = log"${MDC(LogKeys.CLASS_NAME, sortedClassNames.head)}" + sortedClassNames.tail.foreach( + name => message += log", ${MDC(LogKeys.CLASS_NAME, name)}" + ) + if (sortedClassNames.size > 1) { + message + log" are" + } else { + message + log" is" + } + } + + private def notInRTMAllowlistException( + errorType: String, + classNames: Seq[String]): SparkIllegalArgumentException = { + assert(classNames.nonEmpty) + new SparkIllegalArgumentException( + errorClass = "STREAMING_REAL_TIME_MODE.OPERATOR_OR_SINK_NOT_IN_ALLOWLIST", + messageParameters = Map( + "errorType" -> errorType, + "message" -> classNamesString(classNames).message + ) + ) + } + + def checkAllowedSink(sink: Table, throwException: Boolean): Unit = { + if (!allowedSinks.contains(sink.getClass.getName)) { + if (throwException) { + throw notInRTMAllowlistException("sink", Seq(sink.getClass.getName)) + } else { + logWarning( + log"The sink: " + classNamesString(Seq(sink.getClass.getName)) + + log" not in the sink allowlist for Real-Time Mode." + ) + } + } + } + + // Collect ALL nodes whose entire subtree contains RealTimeStreamScanExec. + private def collectRealtimeNodes(root: SparkPlan): Seq[SparkPlan] = { + + def collectNodesWhoseSubtreeHasRTS(n: SparkPlan): (Boolean, List[SparkPlan]) = { + n match { + case _: RealTimeStreamScanExec => + // Subtree has RTS, but we don't collect the RTS node itself. + (true, Nil) + case _ if n.children.isEmpty => + (false, Nil) + case _ => + val kidResults = n.children.map(collectNodesWhoseSubtreeHasRTS) + val anyChildHasRTS = kidResults.exists(_._1) + val collectedKids = kidResults.iterator.flatMap(_._2).toList + val collectedHere = if (anyChildHasRTS) n :: collectedKids else collectedKids + (anyChildHasRTS, collectedHere) + } + } + + collectNodesWhoseSubtreeHasRTS(root)._2 + } + + def checkAllowedPhysicalOperator(operator: SparkPlan, throwException: Boolean): Unit = { + val nodesToCheck = collectRealtimeNodes(operator) + val violations = nodesToCheck + .collect { + case node => + if (allowedOperators.contains(node.getClass.getName)) { + None + } else { + Some(node.getClass.getName) + } + } + .flatten + .distinct + + if (violations.nonEmpty) { + if (throwException) { + throw notInRTMAllowlistException("operator", violations.toSet.toSeq) + } else { + logWarning( + log"The operator(s): " + classNamesString(violations.toSet.toSeq) + + log" not in the operator allowlist for Real-Time Mode." + ) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala index ee7bf67eb9121..86d48b1e88c5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.streaming.{WriteToStream, WriteToStreamStatement} import org.apache.spark.sql.connector.catalog.SupportsWrite import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RealTimeTrigger} import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -48,7 +49,12 @@ object ResolveWriteToStream extends Rule[LogicalPlan] { } if (conf.isUnsupportedOperationCheckEnabled) { - if (s.sink.isInstanceOf[SupportsWrite] && s.isContinuousTrigger) { + if (s.trigger.isInstanceOf[RealTimeTrigger]) { + UnsupportedOperationChecker. + checkAdditionalRealTimeModeConstraints(s.inputQuery, s.outputMode) + } + + if (s.sink.isInstanceOf[SupportsWrite] && s.trigger.isInstanceOf[ContinuousTrigger]) { UnsupportedOperationChecker.checkForContinuous(s.inputQuery, s.outputMode) } else { UnsupportedOperationChecker.checkForStreaming(s.inputQuery, s.outputMode) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeAllowlistSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeAllowlistSuite.scala new file mode 100644 index 0000000000000..4306e5a860120 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeAllowlistSuite.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import scala.concurrent.duration._ + +import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.sql.execution.streaming.LowLatencyMemoryStream +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +class StreamRealTimeModeAllowlistSuite extends StreamRealTimeModeE2ESuiteBase { + import testImplicits._ + + test("rtm source allowlist") { + val query = spark.readStream + .format("rate") + .option("numPartitions", 1) + .load() + .writeStream + .format("console") + .outputMode("update") + .trigger(defaultTrigger) + .start() + + eventually(timeout(60.seconds)) { + checkError( + exception = query.exception.get.getCause.asInstanceOf[SparkIllegalArgumentException], + condition = "STREAMING_REAL_TIME_MODE.INPUT_STREAM_NOT_SUPPORTED", + parameters = Map( + "className" -> + "org.apache.spark.sql.execution.streaming.sources.RateStreamMicroBatchStream") + ) + } + } + + test("rtm operator allowlist") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + val inputData = LowLatencyMemoryStream[Int](2) + val staticDf = spark.range(1, 31, 1, 10).selectExpr("id AS join_key", "id AS join_value") + + val df = inputData.toDF() + .select(col("value").as("key"), col("value").as("value")) + .join(staticDf, col("value") === col("join_key")) + .select( + concat(col("key"), lit("-"), col("value"), lit("-"), col("join_value")).as("output")) + + val query = runStreamingQuery("operation_allowlist", df) + + eventually(timeout(60.seconds)) { + checkError( + exception = query.exception.get.getCause.asInstanceOf[SparkIllegalArgumentException], + condition = "STREAMING_REAL_TIME_MODE.OPERATOR_OR_SINK_NOT_IN_ALLOWLIST", + parameters = Map( + "errorType" -> "operator", + "message" -> ( + "org.apache.spark.sql.execution.SortExec, " + + "org.apache.spark.sql.execution.exchange.ShuffleExchangeExec, " + + "org.apache.spark.sql.execution.joins.SortMergeJoinExec are" + ) + ) + ) + } + } + } + + test("rtm sink allowlist") { + val read = LowLatencyMemoryStream[Int](2) + + val query = read + .toDF() + .writeStream + .format("noop") + .outputMode(OutputMode.Update()) + .trigger(defaultTrigger) + .queryName("rtm_sink_allowlist") + + checkError( + exception = intercept[SparkIllegalArgumentException] { + query.start() + }, + condition = "STREAMING_REAL_TIME_MODE.OPERATOR_OR_SINK_NOT_IN_ALLOWLIST", + parameters = Map( + "errorType" -> "sink", + "message" -> "org.apache.spark.sql.execution.datasources.noop.NoopTable$ is" + )) + + withSQLConf(SQLConf.STREAMING_REAL_TIME_MODE_ALLOWLIST_CHECK.key -> "false") { + val tmp = query.start() + Thread.sleep(5000) + tmp.stop() + } + } + + // TODO(SPARK-54237) : Remove this test after RTM can shuffle to multiple stages + test("repartition not allowed") { + val inputData = LowLatencyMemoryStream[Int](2) + + val df = inputData.toDF() + .select(col("value").as("key")) + .repartition(4, col("key")) + + val query = runStreamingQuery("repartition_allowlist", df) + + eventually(timeout(60.seconds)) { + checkError( + exception = query.exception.get.getCause.asInstanceOf[SparkIllegalArgumentException], + condition = "STREAMING_REAL_TIME_MODE.OPERATOR_OR_SINK_NOT_IN_ALLOWLIST", + parameters = Map( + "errorType" -> "operator", + "message" -> ( + "org.apache.spark.sql.execution.exchange.ShuffleExchangeExec is" + ) + ) + ) + } + } + + // TODO(SPARK-54236) : Remove this test after RTM supports stateful queries + test("stateful queries not allowed") { + val inputData = LowLatencyMemoryStream[Int](2) + + val df = inputData.toDF() + .select(col("value").as("key")) + .groupBy(col("key")) + .count() + .select(concat(col("key"), lit("-"), col("count"))) + + val query = runStreamingQuery("repartition_allowlist", df) + + eventually(timeout(60.seconds)) { + checkError( + exception = query.exception.get.getCause.asInstanceOf[SparkIllegalArgumentException], + condition = "STREAMING_REAL_TIME_MODE.OPERATOR_OR_SINK_NOT_IN_ALLOWLIST", + parameters = Map( + "errorType" -> "operator", + "message" -> ( + "org.apache.spark.sql.execution.aggregate.HashAggregateExec, " + + "org.apache.spark.sql.execution.exchange.ShuffleExchangeExec, " + + "org.apache.spark.sql.execution.streaming" + + ".operators.stateful.StateStoreRestoreExec, " + + "org.apache.spark.sql.execution.streaming.operators.stateful.StateStoreSaveExec are" + ) + ) + ) + } + } +} From b0036509e0fd84deaa9b6e4b2e0952b2121536be Mon Sep 17 00:00:00 2001 From: Jacky Wang Date: Fri, 7 Nov 2025 16:00:03 -0800 Subject: [PATCH 070/400] [SPARK-54191][SDP] Add once to Defineflow Proto ### What changes were proposed in this pull request? Add `once` to Defineflow Proto, to allow creating one-time back-fill flow ### Why are the changes needed? Allow new API argument for SDP flow. ### Does this PR introduce _any_ user-facing change? No, no API change yet ### How was this patch tested? Proto changes Closes #52890 from JiaqiWang18/SPARK-54191-StandaloneFlowDetails-proto. Authored-by: Jacky Wang Signed-off-by: Sandy Ryza (cherry picked from commit 1a724ba6feab1142e66ae5fa12a15f2115eb8e5e) Signed-off-by: Sandy Ryza --- .../resources/error/error-conditions.json | 6 ++ .../sql/connect/proto/pipelines_pb2.py | 70 +++++++++---------- .../sql/connect/proto/pipelines_pb2.pyi | 21 ++++++ .../protobuf/spark/connect/pipelines.proto | 7 ++ .../connect/pipelines/PipelinesHandler.scala | 5 ++ ...SparkDeclarativePipelinesServerSuite.scala | 18 +++++ 6 files changed, 92 insertions(+), 35 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 49a217652791d..e061df065ed57 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1407,6 +1407,12 @@ ], "sqlState" : "42623" }, + "DEFINE_FLOW_ONCE_OPTION_NOT_SUPPORTED" : { + "message" : [ + "Defining a one-time flow with the 'once' option is not supported." + ], + "sqlState" : "0A000" + }, "DESCRIBE_JSON_NOT_EXTENDED" : { "message" : [ "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py index 139de83dc1aaf..0eb77c84b5b57 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.py +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py @@ -42,7 +42,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\xcb"\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12R\n\rdefine_output\x18\x02 \x01(\x0b\x32+.spark.connect.PipelineCommand.DefineOutputH\x00R\x0c\x64\x65\x66ineOutput\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x12\xa1\x01\n*get_query_function_execution_signal_stream\x18\x07 \x01(\x0b\x32\x44.spark.connect.PipelineCommand.GetQueryFunctionExecutionSignalStreamH\x00R%getQueryFunctionExecutionSignalStream\x12\x88\x01\n!define_flow_query_function_result\x18\x08 \x01(\x0b\x32<.spark.connect.PipelineCommand.DefineFlowQueryFunctionResultH\x00R\x1d\x64\x65\x66ineFlowQueryFunctionResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xb4\x02\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x92\n\n\x0c\x44\x65\x66ineOutput\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12$\n\x0boutput_name\x18\x02 \x01(\tH\x02R\noutputName\x88\x01\x01\x12?\n\x0boutput_type\x18\x03 \x01(\x0e\x32\x19.spark.connect.OutputTypeH\x03R\noutputType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x04R\x07\x63omment\x88\x01\x01\x12X\n\x14source_code_location\x18\x05 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12_\n\rtable_details\x18\x06 \x01(\x0b\x32\x38.spark.connect.PipelineCommand.DefineOutput.TableDetailsH\x00R\x0ctableDetails\x12\\\n\x0csink_details\x18\x07 \x01(\x0b\x32\x37.spark.connect.PipelineCommand.DefineOutput.SinkDetailsH\x00R\x0bsinkDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xc0\x03\n\x0cTableDetails\x12x\n\x10table_properties\x18\x01 \x03(\x0b\x32M.spark.connect.PipelineCommand.DefineOutput.TableDetails.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x02 \x03(\tR\rpartitionCols\x12\x1b\n\x06\x66ormat\x18\x03 \x01(\tH\x01R\x06\x66ormat\x88\x01\x01\x12\x43\n\x10schema_data_type\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x0eschemaDataType\x12%\n\rschema_string\x18\x05 \x01(\tH\x00R\x0cschemaString\x12-\n\x12\x63lustering_columns\x18\x06 \x03(\tR\x11\x63lusteringColumns\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x08\n\x06schemaB\t\n\x07_format\x1a\xd1\x01\n\x0bSinkDetails\x12^\n\x07options\x18\x01 \x03(\x0b\x32\x44.spark.connect.PipelineCommand.DefineOutput.SinkDetails.OptionsEntryR\x07options\x12\x1b\n\x06\x66ormat\x18\x02 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0e\n\x0c_output_nameB\x0e\n\x0c_output_typeB\n\n\x08_commentB\x17\n\x15_source_code_location\x1a\xdd\x06\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x03R\x11targetDatasetName\x88\x01\x01\x12Q\n\x08sql_conf\x18\x04 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12 \n\tclient_id\x18\x05 \x01(\tH\x04R\x08\x63lientId\x88\x01\x01\x12X\n\x14source_code_location\x18\x06 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12x\n\x15relation_flow_details\x18\x07 \x01(\x0b\x32\x42.spark.connect.PipelineCommand.DefineFlow.WriteRelationFlowDetailsH\x00R\x13relationFlowDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x61\n\x18WriteRelationFlowDetails\x12\x38\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x08relation\x88\x01\x01\x42\x0b\n\t_relation\x1a:\n\x08Response\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x42\x0c\n\n_flow_nameB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0c\n\n_client_idB\x17\n\x15_source_code_location\x1a\xc2\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x12\x1d\n\x07storage\x18\x06 \x01(\tH\x03R\x07storage\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dryB\n\n\x08_storage\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_text\x1a\x9e\x01\n%GetQueryFunctionExecutionSignalStream\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tclient_id\x18\x02 \x01(\tH\x01R\x08\x63lientId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_client_id\x1a\xdd\x01\n\x1d\x44\x65\x66ineFlowQueryFunctionResult\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x12/\n\x11\x64\x61taflow_graph_id\x18\x02 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x38\n\x08relation\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x02R\x08relation\x88\x01\x01\x42\x0c\n\n_flow_nameB\x14\n\x12_dataflow_graph_idB\x0b\n\t_relationB\x0e\n\x0c\x63ommand_type"\xf0\x05\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x12k\n\x14\x64\x65\x66ine_output_result\x18\x02 \x01(\x0b\x32\x37.spark.connect.PipelineCommandResult.DefineOutputResultH\x00R\x12\x64\x65\x66ineOutputResult\x12\x65\n\x12\x64\x65\x66ine_flow_result\x18\x03 \x01(\x0b\x32\x35.spark.connect.PipelineCommandResult.DefineFlowResultH\x00R\x10\x64\x65\x66ineFlowResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x85\x01\n\x12\x44\x65\x66ineOutputResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifier\x1a\x83\x01\n\x10\x44\x65\x66ineFlowResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifierB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message"\xf1\x01\n\x12SourceCodeLocation\x12 \n\tfile_name\x18\x01 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12$\n\x0bline_number\x18\x02 \x01(\x05H\x01R\nlineNumber\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x03 \x01(\tH\x02R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x0c\n\n_file_nameB\x0e\n\x0c_line_numberB\x12\n\x10_definition_path"E\n$PipelineQueryFunctionExecutionSignal\x12\x1d\n\nflow_names\x18\x01 \x03(\tR\tflowNames"\xd7\x01\n\x17PipelineAnalysisContext\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x02 \x01(\tH\x01R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x14\n\x12_dataflow_graph_idB\x12\n\x10_definition_path*i\n\nOutputType\x12\x1b\n\x17OUTPUT_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x12\x08\n\x04SINK\x10\x04\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\xed"\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12R\n\rdefine_output\x18\x02 \x01(\x0b\x32+.spark.connect.PipelineCommand.DefineOutputH\x00R\x0c\x64\x65\x66ineOutput\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x12\xa1\x01\n*get_query_function_execution_signal_stream\x18\x07 \x01(\x0b\x32\x44.spark.connect.PipelineCommand.GetQueryFunctionExecutionSignalStreamH\x00R%getQueryFunctionExecutionSignalStream\x12\x88\x01\n!define_flow_query_function_result\x18\x08 \x01(\x0b\x32<.spark.connect.PipelineCommand.DefineFlowQueryFunctionResultH\x00R\x1d\x64\x65\x66ineFlowQueryFunctionResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xb4\x02\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x92\n\n\x0c\x44\x65\x66ineOutput\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12$\n\x0boutput_name\x18\x02 \x01(\tH\x02R\noutputName\x88\x01\x01\x12?\n\x0boutput_type\x18\x03 \x01(\x0e\x32\x19.spark.connect.OutputTypeH\x03R\noutputType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x04R\x07\x63omment\x88\x01\x01\x12X\n\x14source_code_location\x18\x05 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12_\n\rtable_details\x18\x06 \x01(\x0b\x32\x38.spark.connect.PipelineCommand.DefineOutput.TableDetailsH\x00R\x0ctableDetails\x12\\\n\x0csink_details\x18\x07 \x01(\x0b\x32\x37.spark.connect.PipelineCommand.DefineOutput.SinkDetailsH\x00R\x0bsinkDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xc0\x03\n\x0cTableDetails\x12x\n\x10table_properties\x18\x01 \x03(\x0b\x32M.spark.connect.PipelineCommand.DefineOutput.TableDetails.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x02 \x03(\tR\rpartitionCols\x12\x1b\n\x06\x66ormat\x18\x03 \x01(\tH\x01R\x06\x66ormat\x88\x01\x01\x12\x43\n\x10schema_data_type\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x0eschemaDataType\x12%\n\rschema_string\x18\x05 \x01(\tH\x00R\x0cschemaString\x12-\n\x12\x63lustering_columns\x18\x06 \x03(\tR\x11\x63lusteringColumns\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x08\n\x06schemaB\t\n\x07_format\x1a\xd1\x01\n\x0bSinkDetails\x12^\n\x07options\x18\x01 \x03(\x0b\x32\x44.spark.connect.PipelineCommand.DefineOutput.SinkDetails.OptionsEntryR\x07options\x12\x1b\n\x06\x66ormat\x18\x02 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0e\n\x0c_output_nameB\x0e\n\x0c_output_typeB\n\n\x08_commentB\x17\n\x15_source_code_location\x1a\xff\x06\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x03R\x11targetDatasetName\x88\x01\x01\x12Q\n\x08sql_conf\x18\x04 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12 \n\tclient_id\x18\x05 \x01(\tH\x04R\x08\x63lientId\x88\x01\x01\x12X\n\x14source_code_location\x18\x06 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12x\n\x15relation_flow_details\x18\x07 \x01(\x0b\x32\x42.spark.connect.PipelineCommand.DefineFlow.WriteRelationFlowDetailsH\x00R\x13relationFlowDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x17\n\x04once\x18\x08 \x01(\x08H\x06R\x04once\x88\x01\x01\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x61\n\x18WriteRelationFlowDetails\x12\x38\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x08relation\x88\x01\x01\x42\x0b\n\t_relation\x1a:\n\x08Response\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x42\x0c\n\n_flow_nameB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0c\n\n_client_idB\x17\n\x15_source_code_locationB\x07\n\x05_once\x1a\xc2\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x12\x1d\n\x07storage\x18\x06 \x01(\tH\x03R\x07storage\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dryB\n\n\x08_storage\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_text\x1a\x9e\x01\n%GetQueryFunctionExecutionSignalStream\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tclient_id\x18\x02 \x01(\tH\x01R\x08\x63lientId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_client_id\x1a\xdd\x01\n\x1d\x44\x65\x66ineFlowQueryFunctionResult\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x12/\n\x11\x64\x61taflow_graph_id\x18\x02 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x38\n\x08relation\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x02R\x08relation\x88\x01\x01\x42\x0c\n\n_flow_nameB\x14\n\x12_dataflow_graph_idB\x0b\n\t_relationB\x0e\n\x0c\x63ommand_type"\xf0\x05\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x12k\n\x14\x64\x65\x66ine_output_result\x18\x02 \x01(\x0b\x32\x37.spark.connect.PipelineCommandResult.DefineOutputResultH\x00R\x12\x64\x65\x66ineOutputResult\x12\x65\n\x12\x64\x65\x66ine_flow_result\x18\x03 \x01(\x0b\x32\x35.spark.connect.PipelineCommandResult.DefineFlowResultH\x00R\x10\x64\x65\x66ineFlowResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x85\x01\n\x12\x44\x65\x66ineOutputResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifier\x1a\x83\x01\n\x10\x44\x65\x66ineFlowResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifierB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message"\xf1\x01\n\x12SourceCodeLocation\x12 \n\tfile_name\x18\x01 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12$\n\x0bline_number\x18\x02 \x01(\x05H\x01R\nlineNumber\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x03 \x01(\tH\x02R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x0c\n\n_file_nameB\x0e\n\x0c_line_numberB\x12\n\x10_definition_path"E\n$PipelineQueryFunctionExecutionSignal\x12\x1d\n\nflow_names\x18\x01 \x03(\tR\tflowNames"\xd7\x01\n\x17PipelineAnalysisContext\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x02 \x01(\tH\x01R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x14\n\x12_dataflow_graph_idB\x12\n\x10_definition_path*i\n\nOutputType\x12\x1b\n\x17OUTPUT_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x12\x08\n\x04SINK\x10\x04\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -69,10 +69,10 @@ ]._serialized_options = b"8\001" _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._loaded_options = None _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_options = b"8\001" - _globals["_OUTPUTTYPE"]._serialized_start = 6105 - _globals["_OUTPUTTYPE"]._serialized_end = 6210 + _globals["_OUTPUTTYPE"]._serialized_start = 6139 + _globals["_OUTPUTTYPE"]._serialized_end = 6244 _globals["_PIPELINECOMMAND"]._serialized_start = 195 - _globals["_PIPELINECOMMAND"]._serialized_end = 4622 + _globals["_PIPELINECOMMAND"]._serialized_end = 4656 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_start = 1129 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_end = 1437 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH_SQLCONFENTRY"]._serialized_start = 1338 @@ -94,37 +94,37 @@ _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS_OPTIONSENTRY"]._serialized_start = 2659 _globals["_PIPELINECOMMAND_DEFINEOUTPUT_SINKDETAILS_OPTIONSENTRY"]._serialized_end = 2717 _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_start = 2833 - _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_end = 3694 + _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_end = 3728 _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_start = 1338 _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_end = 1396 - _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_start = 3427 - _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_end = 3524 - _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_start = 3526 - _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_end = 3584 - _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_start = 3697 - _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_end = 4019 - _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_start = 4022 - _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_end = 4221 - _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_start = 4224 - _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_end = 4382 - _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_start = 4385 - _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_end = 4606 - _globals["_PIPELINECOMMANDRESULT"]._serialized_start = 4625 - _globals["_PIPELINECOMMANDRESULT"]._serialized_end = 5377 - _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_start = 4994 - _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_end = 5092 - _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_start = 5095 - _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_end = 5228 - _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_start = 5231 - _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_end = 5362 - _globals["_PIPELINEEVENTRESULT"]._serialized_start = 5379 - _globals["_PIPELINEEVENTRESULT"]._serialized_end = 5452 - _globals["_PIPELINEEVENT"]._serialized_start = 5454 - _globals["_PIPELINEEVENT"]._serialized_end = 5570 - _globals["_SOURCECODELOCATION"]._serialized_start = 5573 - _globals["_SOURCECODELOCATION"]._serialized_end = 5814 - _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_start = 5816 - _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_end = 5885 - _globals["_PIPELINEANALYSISCONTEXT"]._serialized_start = 5888 - _globals["_PIPELINEANALYSISCONTEXT"]._serialized_end = 6103 + _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_start = 3452 + _globals["_PIPELINECOMMAND_DEFINEFLOW_WRITERELATIONFLOWDETAILS"]._serialized_end = 3549 + _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_start = 3551 + _globals["_PIPELINECOMMAND_DEFINEFLOW_RESPONSE"]._serialized_end = 3609 + _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_start = 3731 + _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_end = 4053 + _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_start = 4056 + _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_end = 4255 + _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_start = 4258 + _globals["_PIPELINECOMMAND_GETQUERYFUNCTIONEXECUTIONSIGNALSTREAM"]._serialized_end = 4416 + _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_start = 4419 + _globals["_PIPELINECOMMAND_DEFINEFLOWQUERYFUNCTIONRESULT"]._serialized_end = 4640 + _globals["_PIPELINECOMMANDRESULT"]._serialized_start = 4659 + _globals["_PIPELINECOMMANDRESULT"]._serialized_end = 5411 + _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_start = 5028 + _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_end = 5126 + _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_start = 5129 + _globals["_PIPELINECOMMANDRESULT_DEFINEOUTPUTRESULT"]._serialized_end = 5262 + _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_start = 5265 + _globals["_PIPELINECOMMANDRESULT_DEFINEFLOWRESULT"]._serialized_end = 5396 + _globals["_PIPELINEEVENTRESULT"]._serialized_start = 5413 + _globals["_PIPELINEEVENTRESULT"]._serialized_end = 5486 + _globals["_PIPELINEEVENT"]._serialized_start = 5488 + _globals["_PIPELINEEVENT"]._serialized_end = 5604 + _globals["_SOURCECODELOCATION"]._serialized_start = 5607 + _globals["_SOURCECODELOCATION"]._serialized_end = 5848 + _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_start = 5850 + _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_end = 5919 + _globals["_PIPELINEANALYSISCONTEXT"]._serialized_start = 5922 + _globals["_PIPELINEANALYSISCONTEXT"]._serialized_end = 6137 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi index 60d131037c99d..e0768a1f6baeb 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi @@ -588,6 +588,7 @@ class PipelineCommand(google.protobuf.message.Message): SOURCE_CODE_LOCATION_FIELD_NUMBER: builtins.int RELATION_FLOW_DETAILS_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int + ONCE_FIELD_NUMBER: builtins.int dataflow_graph_id: builtins.str """The graph to attach this flow to.""" flow_name: builtins.str @@ -612,6 +613,13 @@ class PipelineCommand(google.protobuf.message.Message): ) -> global___PipelineCommand.DefineFlow.WriteRelationFlowDetails: ... @property def extension(self) -> google.protobuf.any_pb2.Any: ... + once: builtins.bool + """If true, define the flow as a one-time flow, such as for backfill. + Set to true changes the flow in two ways: + - The flow is run one time by default. If the pipeline is ran with a full refresh, + the flow will run again. + - The flow function must be a batch DataFrame, not a streaming DataFrame. + """ def __init__( self, *, @@ -624,6 +632,7 @@ class PipelineCommand(google.protobuf.message.Message): relation_flow_details: global___PipelineCommand.DefineFlow.WriteRelationFlowDetails | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., + once: builtins.bool | None = ..., ) -> None: ... def HasField( self, @@ -634,6 +643,8 @@ class PipelineCommand(google.protobuf.message.Message): b"_dataflow_graph_id", "_flow_name", b"_flow_name", + "_once", + b"_once", "_source_code_location", b"_source_code_location", "_target_dataset_name", @@ -648,6 +659,8 @@ class PipelineCommand(google.protobuf.message.Message): b"extension", "flow_name", b"flow_name", + "once", + b"once", "relation_flow_details", b"relation_flow_details", "source_code_location", @@ -665,6 +678,8 @@ class PipelineCommand(google.protobuf.message.Message): b"_dataflow_graph_id", "_flow_name", b"_flow_name", + "_once", + b"_once", "_source_code_location", b"_source_code_location", "_target_dataset_name", @@ -679,6 +694,8 @@ class PipelineCommand(google.protobuf.message.Message): b"extension", "flow_name", b"flow_name", + "once", + b"once", "relation_flow_details", b"relation_flow_details", "source_code_location", @@ -703,6 +720,10 @@ class PipelineCommand(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_flow_name", b"_flow_name"] ) -> typing_extensions.Literal["flow_name"] | None: ... @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_once", b"_once"] + ) -> typing_extensions.Literal["once"] | None: ... + @typing.overload def WhichOneof( self, oneof_group: typing_extensions.Literal[ diff --git a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto index 0fa36f8a15143..a92e24fda9154 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto @@ -152,6 +152,13 @@ message PipelineCommand { optional spark.connect.Relation relation = 1; } + // If true, define the flow as a one-time flow, such as for backfill. + // Set to true changes the flow in two ways: + // - The flow is run one time by default. If the pipeline is ran with a full refresh, + // the flow will run again. + // - The flow function must be a batch DataFrame, not a streaming DataFrame. + optional bool once = 8; + message Response { // Fully qualified flow name that uniquely identify a flow in the Dataflow graph. optional string flow_name = 1; diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala index 0929b07be5237..1a3b0d2231c62 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala @@ -267,6 +267,11 @@ private[connect] object PipelinesHandler extends Logging { flow: proto.PipelineCommand.DefineFlow, transformRelationFunc: Relation => LogicalPlan, sessionHolder: SessionHolder): TableIdentifier = { + if (flow.hasOnce) { + throw new AnalysisException( + "DEFINE_FLOW_ONCE_OPTION_NOT_SUPPORTED", + Map("flowName" -> flow.getFlowName)) + } val dataflowGraphId = flow.getDataflowGraphId val graphElementRegistry = sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala index 9dba27c4525c2..ab60462e87351 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala @@ -71,6 +71,24 @@ class SparkDeclarativePipelinesServerSuite } + gridTest("Define flow 'once' argument not supported")(Seq(true, false)) { onceValue => + val ex = intercept[Exception] { + withRawBlockingStub { implicit stub => + val graphId = createDataflowGraph + sendPlan( + buildPlanFromPipelineCommand( + PipelineCommand + .newBuilder() + .setDefineFlow(DefineFlow + .newBuilder() + .setDataflowGraphId(graphId) + .setOnce(onceValue)) + .build())) + } + } + assert(ex.getMessage.contains("DEFINE_FLOW_ONCE_OPTION_NOT_SUPPORTED")) + } + test( "Cross dependency between SQL dataset and non-SQL dataset is valid and can be registered") { withRawBlockingStub { implicit stub => From 750621e1b1f33bca75b2174e7296292a1d339fc3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 7 Nov 2025 17:49:49 -0800 Subject: [PATCH 071/400] [MINOR][INFRA] Fix the install version comment to Python 3.10 in python-minimum Dockerfile ### What changes were proposed in this pull request? This PR aims to fix a comment with `Python 3.10` instead of `Python 3.9` according to the code inside `python-minimum/Dockerfile`. ### Why are the changes needed? This seems to be a left-over which we missed. ### Does this PR introduce _any_ user-facing change? No. This is a comment change in the infra Dockerfile. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52946 from dongjoon-hyun/minor-minimum. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 4ec6668a688cac095773f6623ac5509d3b1552c5) Signed-off-by: Dongjoon Hyun --- dev/spark-test-image/python-minimum/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index ebafbc69ec4d1..627dccdf34b11 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -66,7 +66,7 @@ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scip # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" -# Install Python 3.9 packages +# Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 RUN python3.10 -m pip install --force $BASIC_PIP_PKGS $CONNECT_PIP_PKGS && \ python3.10 -m pip cache purge From 042eb07aa6de6ed835ed718743fafd2466386c9d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 7 Nov 2025 17:50:58 -0800 Subject: [PATCH 072/400] [SPARK-54245][INFRA] Install `mlflow` at Python 3.14 Docker image ### What changes were proposed in this pull request? This PR aims to install `mlflow` at Python 3.14 Docker image. ### Why are the changes needed? MLFlow 3.6.0 is released with Python 3.14 support via PyArrow 22.0.0. - https://pypi.org/project/mlflow/3.6.0/ Like the other images, we will add the same condition `mlflow>=2.8.1`, but it will install `MLFlow>=3.6.0` effectively. https://github.com/apache/spark/blob/a871ba4464e07cb0229b1289b125bb07da7b6265/dev/spark-test-image/python-313/Dockerfile#L71 ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52948 from dongjoon-hyun/SPARK-54245. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit d16b128d8215a3a6ac75a84d91cf6ef24a55e6e5) Signed-off-by: Dongjoon Hyun --- dev/spark-test-image/python-314/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 236e95e3f52cb..07916fc35a0da 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" From 751f34acd222e0a5c7de13d4b8cc75d78058f8b2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 7 Nov 2025 19:38:52 -0800 Subject: [PATCH 073/400] [SPARK-54046][INFRA] Upgrade PyArrow to 22.0.0 ### What changes were proposed in this pull request? This PR aims to upgrade `PyArrow` to 22.0.0. ### Why are the changes needed? To test against the latest `PyArrow` version. `PyArrow 22.0.0` is the first version to support `Python 3.14`. - https://pypi.org/project/pyarrow/22.0.0/ (2025-10-24) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52748 from dongjoon-hyun/SPARK-54046. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit a32559a3284e299fd86a05dbbeda4bfe73a78b00) Signed-off-by: Dongjoon Hyun --- .github/workflows/python_hosted_runner_test.yml | 2 +- dev/spark-test-image/lint/Dockerfile | 2 +- dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- .../python-311-classic-only/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- dev/spark-test-image/python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- python/pyspark/pandas/tests/io/test_feather.py | 12 +++++++++++- 10 files changed, 20 insertions(+), 10 deletions(-) diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index d55eb1d93799b..b7ee3a8b7113c 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -147,7 +147,7 @@ jobs: run: | python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' - python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ + python${{matrix.python}} -m pip install numpy 'pyarrow>=22.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ python${{matrix.python}} -m pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge - name: List Python packages diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 6686e3808e035..4dfceae63a17a 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -94,7 +94,7 @@ RUN python3.11 -m pip install \ 'pandas' \ 'pandas-stubs==1.2.0.53' \ 'plotly>=4.8' \ - 'pyarrow>=21.0.0' \ + 'pyarrow>=22.0.0' \ 'pytest-mypy-plugins==1.9.3' \ 'pytest==7.1.3' \ && python3.11 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index 06b21bb555bb9..bc9a507853c26 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 -ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=21.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=22.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index 99d231f47bd7e..c318a615b7e05 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-311-classic-only/Dockerfile b/dev/spark-test-image/python-311-classic-only/Dockerfile index 484c940988443..1c5f9a2335787 100644 --- a/dev/spark-test-image/python-311-classic-only/Dockerfile +++ b/dev/spark-test-image/python-311-classic-only/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 pandas==2.3.3 plotly<6.0.0 matplotlib openpyxl memory-profiler>=0.61.0 mlflow>=2.8.1 scipy scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 pandas==2.3.3 plotly<6.0.0 matplotlib openpyxl memory-profiler>=0.61.0 mlflow>=2.8.1 scipy scikit-learn>=1.3.2" ARG TEST_PIP_PKGS="coverage unittest-xml-reporting" # Install Python 3.11 packages diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index a45ab45548480..69d47e62774a5 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index e05fe327a0302..0c8b816f86297 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index d4469a9cd4947..1262089f43e14 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 1d3d73c9feb9c..2e4dde33077da 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ && rm -rf /var/lib/apt/lists/* -ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" diff --git a/python/pyspark/pandas/tests/io/test_feather.py b/python/pyspark/pandas/tests/io/test_feather.py index 3ddf0a2aad925..10638d915c0ef 100644 --- a/python/pyspark/pandas/tests/io/test_feather.py +++ b/python/pyspark/pandas/tests/io/test_feather.py @@ -20,6 +20,7 @@ import sys from pyspark import pandas as ps +from pyspark.loose_version import LooseVersion from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils @@ -35,7 +36,16 @@ def pdf(self): def psdf(self): return ps.from_pandas(self.pdf) - @unittest.skipIf(sys.version_info > (3, 13), "SPARK-54068") + has_arrow_21_or_below = False + try: + import pyarrow as pa + + if LooseVersion(pa.__version__) < LooseVersion("22.0.0"): + has_arrow_21_or_below = True + except ImportError: + pass + + @unittest.skipIf(not has_arrow_21_or_below, "SPARK-54068") def test_to_feather(self): with self.temp_dir() as dirpath: path1 = f"{dirpath}/file1.feather" From 136246c295150d589b70766edb0010321902159f Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Fri, 7 Nov 2025 19:42:48 -0800 Subject: [PATCH 074/400] [SPARK-54243][SQL] Introduce type coercion support for GEOGRAPHY data types ### What changes were proposed in this pull request? Implement least common type (LCT) logic for `GEOGRAPHY` types, as follows: - LCT for GeographyType(`srid_1`) and GeographyType(`srid_1`) is: GeographyType(`srid_1`) - LCT for GeographyType(`srid_1`) and GeographyType(`srid_2`) is: GeographyType(`ANY`) - LCT for GeographyType(`srid_1`) and GeographyType(`ANY`) is: GeographyType(`ANY`) - LCT for GeographyType(`ANY`) and GeographyType(`ANY`) is: GeographyType(`ANY`) In other words, the mixed SRID `GEOGRAPHY` type is the *common type* for all GEOGRAPHY types. ### Why are the changes needed? Introducing LCT and type coercion logic in the geospatial data type system. ### Does this PR introduce _any_ user-facing change? Yes, type coercion is now supported for `GeographyType`. ### How was this patch tested? Added tests for geography type coercion: - `AnsiTypeCoercionSuite` - `TypeCoercionSuite` Added appropriate Scala suite unit tests: - `STExpressionsSuite` Added appropriate end-to-end SQL tests: - `st-functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52944 from uros-db/geo-coercion-geog. Authored-by: Uros Bojanic Signed-off-by: Dongjoon Hyun (cherry picked from commit f2d594de6195b31ccad9de233ad5321be1107a1a) Signed-off-by: Dongjoon Hyun --- .../catalyst/analysis/AnsiTypeCoercion.scala | 4 + .../sql/catalyst/analysis/TypeCoercion.scala | 4 + .../analysis/AnsiTypeCoercionSuite.scala | 7 + .../catalyst/analysis/TypeCoercionSuite.scala | 7 + .../nonansi/st-functions.sql.out | 72 +++++++++ .../analyzer-results/st-functions.sql.out | 72 +++++++++ .../sql-tests/inputs/st-functions.sql | 22 +++ .../results/nonansi/st-functions.sql.out | 81 ++++++++++ .../sql-tests/results/st-functions.sql.out | 81 ++++++++++ .../apache/spark/sql/STExpressionsSuite.scala | 148 ++++++++++++++++++ 10 files changed, 498 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala index 13b554eb53d4d..ea4d04ff6e774 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala @@ -130,6 +130,10 @@ object AnsiTypeCoercion extends TypeCoercionBase { case (t1: YearMonthIntervalType, t2: YearMonthIntervalType) => Some(YearMonthIntervalType(t1.startField.min(t2.startField), t1.endField.max(t2.endField))) + // We allow coercion from GEOGRAPHY() types (i.e. fixed SRID types) to the + // GEOGRAPHY(ANY) type (i.e. mixed SRID type). This coercion is always safe to do. + case (t1: GeographyType, t2: GeographyType) if t1 != t2 => Some(GeographyType("ANY")) + case (t1, t2) => findTypeForComplex(t1, t2, findTightestCommonType) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 3e5f14810935b..9030de9473dea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -98,6 +98,10 @@ object TypeCoercion extends TypeCoercionBase { case (t1: YearMonthIntervalType, t2: YearMonthIntervalType) => Some(YearMonthIntervalType(t1.startField.min(t2.startField), t1.endField.max(t2.endField))) + // We allow coercion from GEOGRAPHY() types (i.e. fixed SRID types) to the + // GEOGRAPHY(ANY) type (i.e. mixed SRID type). This coercion is always safe to do. + case (t1: GeographyType, t2: GeographyType) if t1 != t2 => Some(GeographyType("ANY")) + case (t1, t2) => findTypeForComplex(t1, t2, findTightestCommonType) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala index 42acc38eee2d1..0e5ebcfa313c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala @@ -174,6 +174,13 @@ class AnsiTypeCoercionSuite extends TypeCoercionSuiteBase { widenTest(FloatType, FloatType, Some(FloatType)) widenTest(DoubleType, DoubleType, Some(DoubleType)) + // Geography with same fixed SRIDs. + widenTest(GeographyType(4326), GeographyType(4326), Some(GeographyType(4326))) + // Geography with mixed SRIDs. + widenTest(GeographyType("ANY"), GeographyType("ANY"), Some(GeographyType("ANY"))) + widenTest(GeographyType("ANY"), GeographyType(4326), Some(GeographyType("ANY"))) + widenTest(GeographyType(4326), GeographyType("ANY"), Some(GeographyType("ANY"))) + // Integral mixed with floating point. widenTest(IntegerType, FloatType, Some(DoubleType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 250f20fd09571..0169034c34752 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -597,6 +597,13 @@ class TypeCoercionSuite extends TypeCoercionSuiteBase { widenTest(FloatType, FloatType, Some(FloatType)) widenTest(DoubleType, DoubleType, Some(DoubleType)) + // Geography with same fixed SRIDs. + widenTest(GeographyType(4326), GeographyType(4326), Some(GeographyType(4326))) + // Geography with mixed SRIDs. + widenTest(GeographyType("ANY"), GeographyType("ANY"), Some(GeographyType("ANY"))) + widenTest(GeographyType("ANY"), GeographyType(4326), Some(GeographyType("ANY"))) + widenTest(GeographyType(4326), GeographyType("ANY"), Some(GeographyType("ANY"))) + // Integral mixed with floating point. widenTest(IntegerType, FloatType, Some(FloatType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out index a564c6a329327..71d5a808e686f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out @@ -153,6 +153,78 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT typeof(array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(array(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(array(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(map(a, cast(st_geogfromwkb(wkb#x) as geography(any)), b, cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(map(a, st_geogfromwkb(wkb), b, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata +-- !query analysis +Project [typeof(array(cast(named_struct(g1, st_geogfromwkb(wkb#x), g2, cast(st_geogfromwkb(wkb#x) as geography(any))) as struct), cast(named_struct(g1, cast(st_geogfromwkb(wkb#x) as geography(any)), g2, st_geogfromwkb(wkb#x)) as struct))) AS typeof(array(named_struct(g1, st_geogfromwkb(wkb), g2, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))), named_struct(g1, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)), g2, st_geogfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata +-- !query analysis +Project [typeof(named_struct(a, array(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geogfromwkb(wkb#x) as geography(any))), b, map(g, cast(st_geogfromwkb(wkb#x) as geography(any)), h, cast(st_geogfromwkb(wkb#x) as geography(any))))) AS typeof(named_struct(a, array(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))), b, map(g, st_geogfromwkb(wkb), h, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(nvl(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(nvl(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(nvl2(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geography(any)), st_geogfromwkb(wkb#x))) AS typeof(nvl2(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)), st_geogfromwkb(wkb)))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata +-- !query analysis +Project [typeof(CASE WHEN isnotnull(wkb#x) THEN cast(st_geogfromwkb(wkb#x) as geography(any)) ELSE cast(st_geogfromwkb(wkb#x) as geography(any)) END) AS typeof(CASE WHEN (wkb IS NOT NULL) THEN CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)) ELSE st_geogfromwkb(wkb) END)#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(coalesce(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(coalesce(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(if (isnotnull(wkb#x)) cast(st_geogfromwkb(wkb#x) as geography(any)) else cast(st_geogfromwkb(wkb#x) as geography(any))) AS typeof((IF((wkb IS NOT NULL), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)), st_geogfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out index a564c6a329327..71d5a808e686f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out @@ -153,6 +153,78 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT typeof(array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(array(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(array(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(map(a, cast(st_geogfromwkb(wkb#x) as geography(any)), b, cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(map(a, st_geogfromwkb(wkb), b, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata +-- !query analysis +Project [typeof(array(cast(named_struct(g1, st_geogfromwkb(wkb#x), g2, cast(st_geogfromwkb(wkb#x) as geography(any))) as struct), cast(named_struct(g1, cast(st_geogfromwkb(wkb#x) as geography(any)), g2, st_geogfromwkb(wkb#x)) as struct))) AS typeof(array(named_struct(g1, st_geogfromwkb(wkb), g2, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))), named_struct(g1, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)), g2, st_geogfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata +-- !query analysis +Project [typeof(named_struct(a, array(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geogfromwkb(wkb#x) as geography(any))), b, map(g, cast(st_geogfromwkb(wkb#x) as geography(any)), h, cast(st_geogfromwkb(wkb#x) as geography(any))))) AS typeof(named_struct(a, array(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))), b, map(g, st_geogfromwkb(wkb), h, CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(nvl(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(nvl(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(nvl2(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geography(any)), st_geogfromwkb(wkb#x))) AS typeof(nvl2(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)), st_geogfromwkb(wkb)))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata +-- !query analysis +Project [typeof(CASE WHEN isnotnull(wkb#x) THEN cast(st_geogfromwkb(wkb#x) as geography(any)) ELSE cast(st_geogfromwkb(wkb#x) as geography(any)) END) AS typeof(CASE WHEN (wkb IS NOT NULL) THEN CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)) ELSE st_geogfromwkb(wkb) END)#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query analysis +Project [typeof(coalesce(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geogfromwkb(wkb#x) as geography(any)))) AS typeof(coalesce(st_geogfromwkb(wkb), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(if (isnotnull(wkb#x)) cast(st_geogfromwkb(wkb#x) as geography(any)) else cast(st_geogfromwkb(wkb#x) as geography(any))) AS typeof((IF((wkb IS NOT NULL), CAST(st_geogfromwkb(wkb) AS GEOGRAPHY(ANY)), st_geogfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql index ceda713983050..e1c11bb089b55 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql @@ -28,6 +28,28 @@ SELECT hex(ST_AsBinary(CAST(ST_GeomFromWKB(X'0101000000000000000000f03f000000000 -- Casting GEOMETRY(ANY) to GEOMETRY() is not allowed. SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOMETRY(ANY) AS GEOMETRY(4326)) AS result; +---- Geospatial type coercion + +-- Array +SELECT typeof(array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +-- Map +SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +-- Struct +SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata; +-- Nested +SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata; + +-- NVL +SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +-- NVL2 +SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata; +-- CASE WHEN +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata; +-- COALESCE +SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +-- IF +SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata; + ---- ST reader/writer expressions -- WKB (Well-Known Binary) round-trip tests for GEOGRAPHY and GEOMETRY types. diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out index c01534c64e7c6..71a7d6eaa5dc1 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out @@ -169,6 +169,87 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT typeof(array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +array +array + + +-- !query +SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +map +map + + +-- !query +SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata +-- !query schema +struct +-- !query output +array> +array> + + +-- !query +SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata +-- !query schema +struct +-- !query output +struct,b:map> +struct,b:map> + + +-- !query +SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out index c01534c64e7c6..71a7d6eaa5dc1 100644 --- a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out @@ -169,6 +169,87 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT typeof(array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +array +array + + +-- !query +SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +map +map + + +-- !query +SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata +-- !query schema +struct +-- !query output +array> +array> + + +-- !query +SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata +-- !query schema +struct +-- !query output +struct,b:map> +struct,b:map> + + +-- !query +SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geography(any) +geography(any) + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 79e7ecbdf4c38..6dd8f43471059 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -123,6 +123,154 @@ class STExpressionsSuite } } + /** Geospatial type coercion. */ + + test("CreateArray with GEOGRAPHY literals") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with literals, using geographies with different SRID values. + val geog1 = s"ST_GeogFromWKB(X'$wkbString')" // Literal with fixed SRID (4326). + val geographyType1 = GeographyType(4326) + val geog2 = s"$geog1::GEOGRAPHY(ANY)" // Literal with mixed SRID (ANY). + val geographyType2 = GeographyType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 4326) + + val testCases = Seq( + (s"array($geog1)", geographyType1, Seq(row)), + (s"array($geog2)", geographyType2, Seq(row)), + (s"array($geog1, $geog1)", geographyType1, Seq(row, row)), + (s"array($geog2, $geog2)", geographyType2, Seq(row, row)), + (s"array($geog1, $geog2)", mixedSridGeographyType, Seq(row, row)), + (s"array($geog2, $geog1)", mixedSridGeographyType, Seq(row, row)) + ) + + for ((expr, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $expr", + ArrayType(expectedType) + ) + checkAnswer( + sql(s"WITH t AS (SELECT explode($expr) AS g) SELECT $geo FROM t"), + expectedRows + ) + } + } + + test("CreateArray with GEOGRAPHY columns") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with columns, using geographies with different SRID values. + val geog1 = "ST_GeogFromWKB(wkb)" // Column with fixed SRID (4326). + val geographyType1 = GeographyType(4326) + val geog2 = s"$geog1::GEOGRAPHY(ANY)" // Column with mixed SRID (ANY). + val geographyType2 = GeographyType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 4326) + + val testCases = Seq( + (s"array($geog1)", geographyType1, Seq(row)), + (s"array($geog2)", geographyType2, Seq(row)), + (s"array($geog1, $geog1)", geographyType1, Seq(row, row)), + (s"array($geog2, $geog2)", geographyType2, Seq(row, row)), + (s"array($geog1, $geog2)", mixedSridGeographyType, Seq(row, row)), + (s"array($geog2, $geog1)", mixedSridGeographyType, Seq(row, row)) + ) + + // Test with literal and column, using geographies with different SRID values. + withTable("tbl") { + // Construct and populate the test table. + sql("CREATE TABLE tbl (wkb BINARY)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString')") + + for ((query, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $query FROM tbl", + ArrayType(expectedType) + ) + checkAnswer( + sql(s"WITH t AS (SELECT explode($query) AS g FROM tbl) SELECT $geo FROM t"), + expectedRows + ) + } + } + } + + test("NVL with GEOGRAPHY literals") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with literals, using geographies with different SRID values. + val geog1 = s"ST_GeogFromWKB(X'$wkbString')" // Literal with fixed SRID (4326). + val geographyType1 = GeographyType(4326) + val geog2 = s"$geog1::GEOGRAPHY(ANY)" // Literal with mixed SRID (ANY). + val geographyType2 = GeographyType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 4326) + + val testCases = Seq( + (s"nvl(null, $geog1)", geographyType1, Seq(row)), + (s"nvl($geog1, null)", geographyType1, Seq(row)), + (s"nvl(null, $geog2)", geographyType2, Seq(row)), + (s"nvl($geog2, null)", geographyType2, Seq(row)), + (s"nvl($geog1, $geog1)", geographyType1, Seq(row)), + (s"nvl($geog2, $geog2)", geographyType2, Seq(row)), + (s"nvl($geog1, $geog2)", mixedSridGeographyType, Seq(row)), + (s"nvl($geog2, $geog1)", mixedSridGeographyType, Seq(row)) + ) + + for ((expr, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $expr", + expectedType + ) + checkAnswer( + sql(s"WITH t AS (SELECT $expr AS g) SELECT $geo FROM t"), + expectedRows + ) + } + } + + test("NVL with GEOGRAPHY columns") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with columns, using geographies with different SRID values. + val geog1 = "ST_GeogFromWKB(wkb)" // Column with fixed SRID (4326). + val geographyType1 = GeographyType(4326) + val geog2 = s"$geog1::GEOGRAPHY(ANY)" // Column with mixed SRID (ANY). + val geographyType2 = GeographyType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 4326) + + val testCases = Seq( + (s"nvl(null, $geog1)", geographyType1, Seq(row)), + (s"nvl($geog1, null)", geographyType1, Seq(row)), + (s"nvl(null, $geog2)", geographyType2, Seq(row)), + (s"nvl($geog2, null)", geographyType2, Seq(row)), + (s"nvl($geog1, $geog1)", geographyType1, Seq(row)), + (s"nvl($geog2, $geog2)", geographyType2, Seq(row)), + (s"nvl($geog1, $geog2)", mixedSridGeographyType, Seq(row)), + (s"nvl($geog2, $geog1)", mixedSridGeographyType, Seq(row)) + ) + + // Test with literal and column, using geographies with different SRID values. + withTable("tbl") { + // Construct and populate the test table. + sql("CREATE TABLE tbl (wkb BINARY)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString')") + + for ((query, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $query FROM tbl", + expectedType + ) + checkAnswer( + sql(s"WITH t AS (SELECT $query AS g FROM tbl) SELECT $geo FROM t"), + expectedRows + ) + } + } + } + /** ST reader/writer expressions. */ test("ST_AsBinary") { From a68c2e6ebe35fb69da6b57e6d75130c875766ed6 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Fri, 7 Nov 2025 20:42:39 -0800 Subject: [PATCH 075/400] [SPARK-54246][PYTHON][DOCS] Add the user guide for python worker logging ### What changes were proposed in this pull request? Adds the user guide for python worker logging. ### Why are the changes needed? The documentation for python worker logging will be in the user guide: "Chapter 4: Bug Busting - Debugging PySpark". ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? Yes, asked cursor for the draft. Closes #52949 from ueshin/issues/SPARK-54246/user_guide. Authored-by: Takuya Ueshin Signed-off-by: Dongjoon Hyun (cherry picked from commit b144965891495619302ae23633af14d252b90b8f) Signed-off-by: Dongjoon Hyun --- .../docs/source/user_guide/bugbusting.ipynb | 359 +++++++++++++++++- 1 file changed, 358 insertions(+), 1 deletion(-) diff --git a/python/docs/source/user_guide/bugbusting.ipynb b/python/docs/source/user_guide/bugbusting.ipynb index 8e64bda1175c5..75150596600ef 100644 --- a/python/docs/source/user_guide/bugbusting.ipynb +++ b/python/docs/source/user_guide/bugbusting.ipynb @@ -792,7 +792,7 @@ "id": "09b420ba", "metadata": {}, "source": [ - "## Disply Stacktraces" + "## Display Stacktraces" ] }, { @@ -900,6 +900,363 @@ "See also [Stack Traces](https://spark.apache.org/docs/latest/api/python/development/debugging.html#stack-traces) for more details." ] }, + { + "cell_type": "markdown", + "id": "cff22ba8", + "metadata": {}, + "source": [ + "## Python Worker Logging\n", + "\n", + "
\n", + "Note: This section applies to Spark 4.1\n", + "
\n", + "\n", + "PySpark provides a logging mechanism for Python workers that execute UDFs, UDTFs, Pandas UDFs, and Python data sources. When enabled, all logging output (including `print` statements, standard logging, and exceptions) is captured and made available for querying and analysis.\n", + "\n", + "### Enabling Worker Logging\n", + "\n", + "Worker logging is **disabled by default**. Enable it by setting the Spark SQL configuration:" + ] + }, + { + "cell_type": "code", + "execution_count": 20, + "id": "74786d45", + "metadata": {}, + "outputs": [], + "source": [ + "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")" + ] + }, + { + "cell_type": "markdown", + "id": "0f23fee2", + "metadata": {}, + "source": [ + "### Accessing Logs\n", + "\n", + "All captured logs can be queried as a DataFrame:" + ] + }, + { + "cell_type": "code", + "execution_count": 21, + "id": "9db0c509", + "metadata": {}, + "outputs": [], + "source": [ + "logs = spark.table(\"system.session.python_worker_logs\")" + ] + }, + { + "cell_type": "markdown", + "id": "34bca836", + "metadata": {}, + "source": [ + "The logs DataFrame contains the following columns:\n", + "\n", + "- **ts**: Timestamp of the log entry\n", + "- **level**: Log level (e.g., `\"INFO\"`, `\"WARNING\"`, `\"ERROR\"`)\n", + "- **logger**: Logger name (e.g., custom logger name, `\"stdout\"`, `\"stderr\"`)\n", + "- **msg**: The log message\n", + "- **context**: A map containing contextual information (e.g., `func_name`, `class_name`, custom fields)\n", + "- **exception**: Exception details (if an exception was logged)\n", + "\n", + "### Examples\n", + "\n", + "#### Basic UDF Logging" + ] + }, + { + "cell_type": "code", + "execution_count": 22, + "id": "4cb5bbca", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+------------+\n", + "|my_udf(text)|\n", + "+------------+\n", + "| HELLO|\n", + "| WORLD|\n", + "+------------+\n", + "\n", + "+-------+------------------------+----------------+---------------------+\n", + "|level |msg |logger |context |\n", + "+-------+------------------------+----------------+---------------------+\n", + "|INFO |Processing value: hello |my_custom_logger|{func_name -> my_udf}|\n", + "|WARNING|This is a warning |my_custom_logger|{func_name -> my_udf}|\n", + "|INFO |This is a stdout message|stdout |{func_name -> my_udf}|\n", + "|ERROR |This is a stderr message|stderr |{func_name -> my_udf}|\n", + "|INFO |Processing value: world |my_custom_logger|{func_name -> my_udf}|\n", + "|WARNING|This is a warning |my_custom_logger|{func_name -> my_udf}|\n", + "|INFO |This is a stdout message|stdout |{func_name -> my_udf}|\n", + "|ERROR |This is a stderr message|stderr |{func_name -> my_udf}|\n", + "+-------+------------------------+----------------+---------------------+\n", + "\n" + ] + } + ], + "source": [ + "from pyspark.sql.functions import udf\n", + "import logging\n", + "import sys\n", + "\n", + "@udf(\"string\")\n", + "def my_udf(value):\n", + " logger = logging.getLogger(\"my_custom_logger\")\n", + " logger.setLevel(logging.INFO) # Set level to INFO to capture info messages\n", + " logger.info(f\"Processing value: {value}\")\n", + " logger.warning(\"This is a warning\")\n", + " print(\"This is a stdout message\") # INFO level, logger=stdout\n", + " print(\"This is a stderr message\", file=sys.stderr) # ERROR level, logger=stderr\n", + " return value.upper()\n", + "\n", + "# Enable logging and execute\n", + "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")\n", + "df = spark.createDataFrame([(\"hello\",), (\"world\",)], [\"text\"])\n", + "df.select(my_udf(\"text\")).show()\n", + "\n", + "# Query the logs\n", + "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs.select(\"level\", \"msg\", \"logger\", \"context\").show(truncate=False)" + ] + }, + { + "cell_type": "markdown", + "id": "15a80ffb", + "metadata": {}, + "source": [ + "#### Logging with Custom Context\n", + "\n", + "You can add custom context information to your logs:" + ] + }, + { + "cell_type": "code", + "execution_count": 23, + "id": "427a06c5", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+--------------------+\n", + "|contextual_udf(test)|\n", + "+--------------------+\n", + "| test|\n", + "+--------------------+\n", + "\n", + "+-----------------------------+---------------------------------------------------------------------+\n", + "|msg |context |\n", + "+-----------------------------+---------------------------------------------------------------------+\n", + "|Processing with extra context|{func_name -> contextual_udf, user_id -> 123, operation -> transform}|\n", + "+-----------------------------+---------------------------------------------------------------------+\n", + "\n" + ] + } + ], + "source": [ + "from pyspark.sql.functions import lit, udf\n", + "import logging\n", + "\n", + "@udf(\"string\")\n", + "def contextual_udf(value):\n", + " logger = logging.getLogger(\"contextual\")\n", + " logger.warning(\n", + " \"Processing with extra context\",\n", + " extra={\"context\": {\"user_id\": 123, \"operation\": \"transform\"}}\n", + " )\n", + " return value\n", + "\n", + "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")\n", + "spark.range(1).select(contextual_udf(lit(\"test\"))).show()\n", + "\n", + "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs.filter(\"logger = 'contextual'\").select(\"msg\", \"context\").show(truncate=False)" + ] + }, + { + "cell_type": "markdown", + "id": "a19db296", + "metadata": {}, + "source": [ + "The context includes both automatic fields (like `func_name`) and custom fields (like `user_id`, `operation`).\n", + "\n", + "#### Exception Logging\n", + "\n", + "Exceptions are automatically captured with full stack traces:" + ] + }, + { + "cell_type": "code", + "execution_count": 24, + "id": "3ab34a4c", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+------------------+\n", + "|failing_udf(value)|\n", + "+------------------+\n", + "| -1|\n", + "| 20|\n", + "+------------------+\n", + "\n", + "+-------------------------+----------------------------------------------------------------------------------------------------------------------------------------------+\n", + "|msg |exception |\n", + "+-------------------------+----------------------------------------------------------------------------------------------------------------------------------------------+\n", + "|Division by zero occurred|{ZeroDivisionError, division by zero, [{NULL, failing_udf, /var/folders/r8/0v7zwfbd59q4ym2gn6kxjq8h0000gp/T/ipykernel_79089/916837455.py, 8}]}|\n", + "+-------------------------+----------------------------------------------------------------------------------------------------------------------------------------------+\n", + "\n" + ] + } + ], + "source": [ + "from pyspark.sql.functions import udf\n", + "import logging\n", + "\n", + "@udf(\"int\")\n", + "def failing_udf(x):\n", + " logger = logging.getLogger(\"error_handler\")\n", + " try:\n", + " result = 100 / x\n", + " except ZeroDivisionError:\n", + " logger.exception(\"Division by zero occurred\")\n", + " return -1\n", + " return int(result)\n", + "\n", + "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")\n", + "spark.createDataFrame([(0,), (5,)], [\"value\"]).select(failing_udf(\"value\")).show()\n", + "\n", + "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs.filter(\"logger = 'error_handler'\").select(\"msg\", \"exception\").show(truncate=False)" + ] + }, + { + "cell_type": "markdown", + "id": "e54f6ac3", + "metadata": {}, + "source": [ + "#### UDTF and Python Data Source Logging\n", + "\n", + "Worker logging also works with UDTFs and Python Data Sources, capturing both the class and function names:" + ] + }, + { + "cell_type": "code", + "execution_count": 25, + "id": "02d454b0", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+-----------+-----+------+\n", + "| text| word|length|\n", + "+-----------+-----+------+\n", + "|hello world|hello| 5|\n", + "|hello world|world| 5|\n", + "+-----------+-----+------+\n", + "\n", + "+-----------------------------+---------------------------------------------------------------------+\n", + "|msg |context |\n", + "+-----------------------------+---------------------------------------------------------------------+\n", + "|Processing 2 words |{func_name -> eval, class_name -> WordSplitter} |\n", + "+-----------------------------+---------------------------------------------------------------------+\n", + "\n" + ] + } + ], + "source": [ + "from pyspark.sql.functions import col, udtf\n", + "import logging\n", + "\n", + "@udtf(returnType=\"word: string, length: int\")\n", + "class WordSplitter:\n", + " def eval(self, text: str):\n", + " logger = logging.getLogger(\"udtf_logger\")\n", + " logger.setLevel(logging.INFO) # Set level to INFO to capture info messages\n", + " words = text.split()\n", + " logger.info(f\"Processing {len(words)} words\")\n", + " for word in words:\n", + " yield (word, len(word))\n", + "\n", + "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")\n", + "df = spark.createDataFrame([(\"hello world\",)], [\"text\"])\n", + "df.lateralJoin(WordSplitter(col(\"text\").outer())).show()\n", + "\n", + "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs.filter(\"logger = 'udtf_logger'\").select(\"msg\", \"context\").show(truncate=False)" + ] + }, + { + "cell_type": "markdown", + "id": "9d4c119b", + "metadata": {}, + "source": [ + "### Querying and Analyzing Logs\n", + "\n", + "You can use standard DataFrame operations to analyze logs:" + ] + }, + { + "cell_type": "code", + "execution_count": 26, + "id": "5b061011", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+-------+-----+\n", + "| level|count|\n", + "+-------+-----+\n", + "| INFO| 5|\n", + "|WARNING| 3|\n", + "| ERROR| 3|\n", + "+-------+-----+\n", + "\n", + "...\n", + "\n" + ] + } + ], + "source": [ + "logs = spark.table(\"system.session.python_worker_logs\")\n", + "\n", + "# Count logs by level\n", + "logs.groupBy(\"level\").count().show()\n", + "\n", + "# Find all errors\n", + "logs.filter(\"level = 'ERROR'\").show()\n", + "\n", + "# Logs from a specific function\n", + "logs.filter(\"context.func_name = 'my_udf'\").show()\n", + "\n", + "# Logs with exceptions\n", + "logs.filter(\"exception is not null\").show()\n", + "\n", + "# Time-based analysis\n", + "logs.orderBy(\"ts\").show()" + ] + }, + { + "cell_type": "markdown", + "id": "7eaa72b9", + "metadata": {}, + "source": [ + "\n" + ] + }, { "attachments": {}, "cell_type": "markdown", From 13e07fc6a85f7fca1a1d925f91bba325312b57ef Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Sat, 8 Nov 2025 07:30:25 -0800 Subject: [PATCH 076/400] [SPARK-54244][GEO][SQL] Introduce type coercion support for GEOMETRY data types ### What changes were proposed in this pull request? Implement least common type (LCT) logic for `GEOMETRY` types, as follows: - LCT for GeometryType(`srid_1`) and GeometryType(`srid_1`) is: GeometryType(`srid_1`) - LCT for GeometryType(`srid_1`) and GeometryType(`srid_2`) is: GeometryType(`ANY`) - LCT for GeometryType(`srid_1`) and GeometryType(`ANY`) is: GeometryType(`ANY`) - LCT for GeometryType(`ANY`) and GeometryType(`ANY`) is: GeometryType(`ANY`) In other words, the mixed SRID `GEOMETRY` type is the *common type* for all GEOMETRY types. ### Why are the changes needed? Introducing LCT and type coercion logic in the geospatial data type system. ### Does this PR introduce _any_ user-facing change? Yes, type coercion is now supported for `GeometryType`. ### How was this patch tested? Added tests for geometry type coercion: - `AnsiTypeCoercionSuite` - `TypeCoercionSuite` Added appropriate Scala suite unit tests: - `STExpressionsSuite` Added appropriate end-to-end SQL tests: - `st-functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52945 from uros-db/geo-coercion-geom. Authored-by: Uros Bojanic Signed-off-by: Dongjoon Hyun (cherry picked from commit 9b68d3c92b4c7bff7abcb8bb5fe540f2bfefd705) Signed-off-by: Dongjoon Hyun --- .../catalyst/analysis/AnsiTypeCoercion.scala | 3 + .../sql/catalyst/analysis/TypeCoercion.scala | 3 + .../analysis/AnsiTypeCoercionSuite.scala | 12 ++ .../catalyst/analysis/TypeCoercionSuite.scala | 12 ++ .../nonansi/st-functions.sql.out | 72 +++++++++ .../analyzer-results/st-functions.sql.out | 72 +++++++++ .../sql-tests/inputs/st-functions.sql | 9 ++ .../results/nonansi/st-functions.sql.out | 81 ++++++++++ .../sql-tests/results/st-functions.sql.out | 81 ++++++++++ .../apache/spark/sql/STExpressionsSuite.scala | 146 ++++++++++++++++++ 10 files changed, 491 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala index ea4d04ff6e774..e23e7561f0e36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala @@ -133,6 +133,9 @@ object AnsiTypeCoercion extends TypeCoercionBase { // We allow coercion from GEOGRAPHY() types (i.e. fixed SRID types) to the // GEOGRAPHY(ANY) type (i.e. mixed SRID type). This coercion is always safe to do. case (t1: GeographyType, t2: GeographyType) if t1 != t2 => Some(GeographyType("ANY")) + // We allow coercion from GEOMETRY() types (i.e. fixed SRID types) to the + // GEOMETRY(ANY) type (i.e. mixed SRID type). This coercion is always safe to do. + case (t1: GeometryType, t2: GeometryType) if t1 != t2 => Some(GeometryType("ANY")) case (t1, t2) => findTypeForComplex(t1, t2, findTightestCommonType) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 9030de9473dea..ce387ef397aca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -101,6 +101,9 @@ object TypeCoercion extends TypeCoercionBase { // We allow coercion from GEOGRAPHY() types (i.e. fixed SRID types) to the // GEOGRAPHY(ANY) type (i.e. mixed SRID type). This coercion is always safe to do. case (t1: GeographyType, t2: GeographyType) if t1 != t2 => Some(GeographyType("ANY")) + // We allow coercion from GEOMETRY() types (i.e. fixed SRID types) to the + // GEOMETRY(ANY) type (i.e. mixed SRID type). This coercion is always safe to do. + case (t1: GeometryType, t2: GeometryType) if t1 != t2 => Some(GeometryType("ANY")) case (t1, t2) => findTypeForComplex(t1, t2, findTightestCommonType) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala index 0e5ebcfa313c6..fa5027ce259d5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala @@ -180,6 +180,18 @@ class AnsiTypeCoercionSuite extends TypeCoercionSuiteBase { widenTest(GeographyType("ANY"), GeographyType("ANY"), Some(GeographyType("ANY"))) widenTest(GeographyType("ANY"), GeographyType(4326), Some(GeographyType("ANY"))) widenTest(GeographyType(4326), GeographyType("ANY"), Some(GeographyType("ANY"))) + // Geometry with same fixed SRIDs. + widenTest(GeometryType(0), GeometryType(0), Some(GeometryType(0))) + widenTest(GeometryType(3857), GeometryType(3857), Some(GeometryType(3857))) + widenTest(GeometryType(4326), GeometryType(4326), Some(GeometryType(4326))) + // Geometry with different fixed SRIDs. + widenTest(GeometryType(0), GeometryType(3857), Some(GeometryType("ANY"))) + widenTest(GeometryType(3857), GeometryType(4326), Some(GeometryType("ANY"))) + widenTest(GeometryType(4326), GeometryType(0), Some(GeometryType("ANY"))) + // Geometry with mixed SRIDs. + widenTest(GeometryType("ANY"), GeometryType("ANY"), Some(GeometryType("ANY"))) + widenTest(GeometryType("ANY"), GeometryType(4326), Some(GeometryType("ANY"))) + widenTest(GeometryType(4326), GeometryType("ANY"), Some(GeometryType("ANY"))) // Integral mixed with floating point. widenTest(IntegerType, FloatType, Some(DoubleType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 0169034c34752..e6a9690ad7570 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -603,6 +603,18 @@ class TypeCoercionSuite extends TypeCoercionSuiteBase { widenTest(GeographyType("ANY"), GeographyType("ANY"), Some(GeographyType("ANY"))) widenTest(GeographyType("ANY"), GeographyType(4326), Some(GeographyType("ANY"))) widenTest(GeographyType(4326), GeographyType("ANY"), Some(GeographyType("ANY"))) + // Geometry with same fixed SRIDs. + widenTest(GeometryType(0), GeometryType(0), Some(GeometryType(0))) + widenTest(GeometryType(3857), GeometryType(3857), Some(GeometryType(3857))) + widenTest(GeometryType(4326), GeometryType(4326), Some(GeometryType(4326))) + // Geometry with different fixed SRIDs. + widenTest(GeometryType(0), GeometryType(3857), Some(GeometryType("ANY"))) + widenTest(GeometryType(3857), GeometryType(4326), Some(GeometryType("ANY"))) + widenTest(GeometryType(4326), GeometryType(0), Some(GeometryType("ANY"))) + // Geometry with mixed SRIDs. + widenTest(GeometryType("ANY"), GeometryType("ANY"), Some(GeometryType("ANY"))) + widenTest(GeometryType("ANY"), GeometryType(4326), Some(GeometryType("ANY"))) + widenTest(GeometryType(4326), GeometryType("ANY"), Some(GeometryType("ANY"))) // Integral mixed with floating point. widenTest(IntegerType, FloatType, Some(FloatType)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out index 71d5a808e686f..c86d2454d759c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/st-functions.sql.out @@ -161,6 +161,14 @@ Project [typeof(array(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geo +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(array(cast(st_geomfromwkb(wkb#x) as geometry(any)), cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(array(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query analysis @@ -169,6 +177,14 @@ Project [typeof(map(a, cast(st_geogfromwkb(wkb#x) as geography(any)), b, cast(st +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(map('a', ST_GeomFromWKB(wkb), 'b', ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(map(a, cast(st_geomfromwkb(wkb#x) as geometry(any)), b, cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(map(a, st_geomfromwkb(wkb), b, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata -- !query analysis @@ -177,6 +193,14 @@ Project [typeof(array(cast(named_struct(g1, st_geogfromwkb(wkb#x), g2, cast(st_g +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(array(named_struct('g1', ST_GeomFromWKB(wkb), 'g2', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), named_struct('g1', ST_GeomFromWKB(wkb)::GEOMETRY(ANY), 'g2', ST_GeomFromWKB(wkb)))) FROM geodata +-- !query analysis +Project [typeof(array(cast(named_struct(g1, st_geomfromwkb(wkb#x), g2, cast(st_geomfromwkb(wkb#x) as geometry(any))) as struct), cast(named_struct(g1, cast(st_geomfromwkb(wkb#x) as geometry(any)), g2, st_geomfromwkb(wkb#x)) as struct))) AS typeof(array(named_struct(g1, st_geomfromwkb(wkb), g2, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))), named_struct(g1, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)), g2, st_geomfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata -- !query analysis @@ -185,6 +209,14 @@ Project [typeof(named_struct(a, array(cast(st_geogfromwkb(wkb#x) as geography(an +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(named_struct('a', array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), 'b', map('g', ST_GeomFromWKB(wkb), 'h', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)))) FROM geodata +-- !query analysis +Project [typeof(named_struct(a, array(cast(st_geomfromwkb(wkb#x) as geometry(any)), cast(st_geomfromwkb(wkb#x) as geometry(any))), b, map(g, cast(st_geomfromwkb(wkb#x) as geometry(any)), h, cast(st_geomfromwkb(wkb#x) as geometry(any))))) AS typeof(named_struct(a, array(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))), b, map(g, st_geomfromwkb(wkb), h, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query analysis @@ -193,6 +225,14 @@ Project [typeof(nvl(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geograp +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(nvl(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(nvl(st_geomfromwkb(wkb#x), cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(nvl(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query analysis @@ -201,6 +241,14 @@ Project [typeof(nvl2(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geogra +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(nvl2(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(nvl2(st_geomfromwkb(wkb#x), cast(st_geomfromwkb(wkb#x) as geometry(any)), st_geomfromwkb(wkb#x))) AS typeof(nvl2(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)), st_geomfromwkb(wkb)))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata -- !query analysis @@ -209,6 +257,14 @@ Project [typeof(CASE WHEN isnotnull(wkb#x) THEN cast(st_geogfromwkb(wkb#x) as ge +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeomFromWKB(wkb)::GEOMETRY(ANY) ELSE ST_GeomFromWKB(wkb) END) FROM geodata +-- !query analysis +Project [typeof(CASE WHEN isnotnull(wkb#x) THEN cast(st_geomfromwkb(wkb#x) as geometry(any)) ELSE cast(st_geomfromwkb(wkb#x) as geometry(any)) END) AS typeof(CASE WHEN (wkb IS NOT NULL) THEN CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)) ELSE st_geomfromwkb(wkb) END)#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query analysis @@ -217,6 +273,14 @@ Project [typeof(coalesce(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_ +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(coalesce(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(coalesce(cast(st_geomfromwkb(wkb#x) as geometry(any)), cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(coalesce(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query analysis @@ -225,6 +289,14 @@ Project [typeof(if (isnotnull(wkb#x)) cast(st_geogfromwkb(wkb#x) as geography(an +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(if (isnotnull(wkb#x)) cast(st_geomfromwkb(wkb#x) as geometry(any)) else cast(st_geomfromwkb(wkb#x) as geometry(any))) AS typeof((IF((wkb IS NOT NULL), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)), st_geomfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out index 71d5a808e686f..c86d2454d759c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/st-functions.sql.out @@ -161,6 +161,14 @@ Project [typeof(array(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_geo +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(array(cast(st_geomfromwkb(wkb#x) as geometry(any)), cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(array(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query analysis @@ -169,6 +177,14 @@ Project [typeof(map(a, cast(st_geogfromwkb(wkb#x) as geography(any)), b, cast(st +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(map('a', ST_GeomFromWKB(wkb), 'b', ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(map(a, cast(st_geomfromwkb(wkb#x) as geometry(any)), b, cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(map(a, st_geomfromwkb(wkb), b, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata -- !query analysis @@ -177,6 +193,14 @@ Project [typeof(array(cast(named_struct(g1, st_geogfromwkb(wkb#x), g2, cast(st_g +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(array(named_struct('g1', ST_GeomFromWKB(wkb), 'g2', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), named_struct('g1', ST_GeomFromWKB(wkb)::GEOMETRY(ANY), 'g2', ST_GeomFromWKB(wkb)))) FROM geodata +-- !query analysis +Project [typeof(array(cast(named_struct(g1, st_geomfromwkb(wkb#x), g2, cast(st_geomfromwkb(wkb#x) as geometry(any))) as struct), cast(named_struct(g1, cast(st_geomfromwkb(wkb#x) as geometry(any)), g2, st_geomfromwkb(wkb#x)) as struct))) AS typeof(array(named_struct(g1, st_geomfromwkb(wkb), g2, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))), named_struct(g1, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)), g2, st_geomfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata -- !query analysis @@ -185,6 +209,14 @@ Project [typeof(named_struct(a, array(cast(st_geogfromwkb(wkb#x) as geography(an +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(named_struct('a', array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), 'b', map('g', ST_GeomFromWKB(wkb), 'h', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)))) FROM geodata +-- !query analysis +Project [typeof(named_struct(a, array(cast(st_geomfromwkb(wkb#x) as geometry(any)), cast(st_geomfromwkb(wkb#x) as geometry(any))), b, map(g, cast(st_geomfromwkb(wkb#x) as geometry(any)), h, cast(st_geomfromwkb(wkb#x) as geometry(any))))) AS typeof(named_struct(a, array(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))), b, map(g, st_geomfromwkb(wkb), h, CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query analysis @@ -193,6 +225,14 @@ Project [typeof(nvl(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geograp +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(nvl(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(nvl(st_geomfromwkb(wkb#x), cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(nvl(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query analysis @@ -201,6 +241,14 @@ Project [typeof(nvl2(st_geogfromwkb(wkb#x), cast(st_geogfromwkb(wkb#x) as geogra +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(nvl2(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(nvl2(st_geomfromwkb(wkb#x), cast(st_geomfromwkb(wkb#x) as geometry(any)), st_geomfromwkb(wkb#x))) AS typeof(nvl2(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)), st_geomfromwkb(wkb)))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata -- !query analysis @@ -209,6 +257,14 @@ Project [typeof(CASE WHEN isnotnull(wkb#x) THEN cast(st_geogfromwkb(wkb#x) as ge +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeomFromWKB(wkb)::GEOMETRY(ANY) ELSE ST_GeomFromWKB(wkb) END) FROM geodata +-- !query analysis +Project [typeof(CASE WHEN isnotnull(wkb#x) THEN cast(st_geomfromwkb(wkb#x) as geometry(any)) ELSE cast(st_geomfromwkb(wkb#x) as geometry(any)) END) AS typeof(CASE WHEN (wkb IS NOT NULL) THEN CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)) ELSE st_geomfromwkb(wkb) END)#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query analysis @@ -217,6 +273,14 @@ Project [typeof(coalesce(cast(st_geogfromwkb(wkb#x) as geography(any)), cast(st_ +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(coalesce(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query analysis +Project [typeof(coalesce(cast(st_geomfromwkb(wkb#x) as geometry(any)), cast(st_geomfromwkb(wkb#x) as geometry(any)))) AS typeof(coalesce(st_geomfromwkb(wkb), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query analysis @@ -225,6 +289,14 @@ Project [typeof(if (isnotnull(wkb#x)) cast(st_geogfromwkb(wkb#x) as geography(an +- Relation spark_catalog.default.geodata[wkb#x] parquet +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query analysis +Project [typeof(if (isnotnull(wkb#x)) cast(st_geomfromwkb(wkb#x) as geometry(any)) else cast(st_geomfromwkb(wkb#x) as geometry(any))) AS typeof((IF((wkb IS NOT NULL), CAST(st_geomfromwkb(wkb) AS GEOMETRY(ANY)), st_geomfromwkb(wkb))))#x] ++- SubqueryAlias spark_catalog.default.geodata + +- Relation spark_catalog.default.geodata[wkb#x] parquet + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql index e1c11bb089b55..70fbdef533303 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/st-functions.sql @@ -32,23 +32,32 @@ SELECT CAST(ST_GeomFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOME -- Array SELECT typeof(array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +SELECT typeof(array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata; -- Map SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +SELECT typeof(map('a', ST_GeomFromWKB(wkb), 'b', ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata; -- Struct SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata; +SELECT typeof(array(named_struct('g1', ST_GeomFromWKB(wkb), 'g2', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), named_struct('g1', ST_GeomFromWKB(wkb)::GEOMETRY(ANY), 'g2', ST_GeomFromWKB(wkb)))) FROM geodata; -- Nested SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata; +SELECT typeof(named_struct('a', array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), 'b', map('g', ST_GeomFromWKB(wkb), 'h', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)))) FROM geodata; -- NVL SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +SELECT typeof(nvl(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata; -- NVL2 SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata; +SELECT typeof(nvl2(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata; -- CASE WHEN SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata; +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeomFromWKB(wkb)::GEOMETRY(ANY) ELSE ST_GeomFromWKB(wkb) END) FROM geodata; -- COALESCE SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata; +SELECT typeof(coalesce(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata; -- IF SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata; +SELECT typeof(IF(wkb IS NOT NULL, ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata; ---- ST reader/writer expressions diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out index 71a7d6eaa5dc1..95e7412bfea4c 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/st-functions.sql.out @@ -178,6 +178,15 @@ array array +-- !query +SELECT typeof(array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +array +array + + -- !query SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query schema @@ -187,6 +196,15 @@ map map +-- !query +SELECT typeof(map('a', ST_GeomFromWKB(wkb), 'b', ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +map +map + + -- !query SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata -- !query schema @@ -196,6 +214,15 @@ array> array> +-- !query +SELECT typeof(array(named_struct('g1', ST_GeomFromWKB(wkb), 'g2', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), named_struct('g1', ST_GeomFromWKB(wkb)::GEOMETRY(ANY), 'g2', ST_GeomFromWKB(wkb)))) FROM geodata +-- !query schema +struct +-- !query output +array> +array> + + -- !query SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata -- !query schema @@ -205,6 +232,15 @@ struct,b:map> struct,b:map> +-- !query +SELECT typeof(named_struct('a', array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), 'b', map('g', ST_GeomFromWKB(wkb), 'h', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)))) FROM geodata +-- !query schema +struct +-- !query output +struct,b:map> +struct,b:map> + + -- !query SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query schema @@ -214,6 +250,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(nvl(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query schema @@ -223,6 +268,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(nvl2(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata -- !query schema @@ -232,6 +286,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeomFromWKB(wkb)::GEOMETRY(ANY) ELSE ST_GeomFromWKB(wkb) END) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query schema @@ -241,6 +304,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(coalesce(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query schema @@ -250,6 +322,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out index 71a7d6eaa5dc1..95e7412bfea4c 100644 --- a/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/st-functions.sql.out @@ -178,6 +178,15 @@ array array +-- !query +SELECT typeof(array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +array +array + + -- !query SELECT typeof(map('a', ST_GeogFromWKB(wkb), 'b', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query schema @@ -187,6 +196,15 @@ map map +-- !query +SELECT typeof(map('a', ST_GeomFromWKB(wkb), 'b', ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +map +map + + -- !query SELECT typeof(array(named_struct('g1', ST_GeogFromWKB(wkb), 'g2', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), named_struct('g1', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), 'g2', ST_GeogFromWKB(wkb)))) FROM geodata -- !query schema @@ -196,6 +214,15 @@ array> array> +-- !query +SELECT typeof(array(named_struct('g1', ST_GeomFromWKB(wkb), 'g2', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), named_struct('g1', ST_GeomFromWKB(wkb)::GEOMETRY(ANY), 'g2', ST_GeomFromWKB(wkb)))) FROM geodata +-- !query schema +struct +-- !query output +array> +array> + + -- !query SELECT typeof(named_struct('a', array(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)), 'b', map('g', ST_GeogFromWKB(wkb), 'h', ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY)))) FROM geodata -- !query schema @@ -205,6 +232,15 @@ struct,b:map> struct,b:map> +-- !query +SELECT typeof(named_struct('a', array(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY)), 'b', map('g', ST_GeomFromWKB(wkb), 'h', ST_GeomFromWKB(wkb)::GEOMETRY(ANY)))) FROM geodata +-- !query schema +struct +-- !query output +struct,b:map> +struct,b:map> + + -- !query SELECT typeof(nvl(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query schema @@ -214,6 +250,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(nvl(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(nvl2(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query schema @@ -223,6 +268,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(nvl2(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY) ELSE ST_GeogFromWKB(wkb) END) FROM geodata -- !query schema @@ -232,6 +286,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(CASE WHEN wkb IS NOT NULL THEN ST_GeomFromWKB(wkb)::GEOMETRY(ANY) ELSE ST_GeomFromWKB(wkb) END) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(coalesce(ST_GeogFromWKB(wkb), ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY))) FROM geodata -- !query schema @@ -241,6 +304,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(coalesce(ST_GeomFromWKB(wkb), ST_GeomFromWKB(wkb)::GEOMETRY(ANY))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT typeof(IF(wkb IS NOT NULL, ST_GeogFromWKB(wkb)::GEOGRAPHY(ANY), ST_GeogFromWKB(wkb))) FROM geodata -- !query schema @@ -250,6 +322,15 @@ geography(any) geography(any) +-- !query +SELECT typeof(IF(wkb IS NOT NULL, ST_GeomFromWKB(wkb)::GEOMETRY(ANY), ST_GeomFromWKB(wkb))) FROM geodata +-- !query schema +struct +-- !query output +geometry(any) +geometry(any) + + -- !query SELECT hex(ST_AsBinary(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040'))) AS result -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 6dd8f43471059..4f0d567fccc47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -271,6 +271,152 @@ class STExpressionsSuite } } + test("CreateArray with GEOMETRY literals") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with literals, using geometries with different SRID values. + val geom1 = s"ST_GeomFromWKB(X'$wkbString')" // Literal with fixed SRID (0). + val geometryType1 = GeometryType(0) + val geom2 = s"$geom1::GEOMETRY(ANY)" // Literal with mixed SRID (ANY). + val geometryType2 = GeometryType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 0) + + val testCases = Seq( + (s"array($geom1)", geometryType1, Seq(row)), + (s"array($geom2)", geometryType2, Seq(row)), + (s"array($geom1, $geom1)", geometryType1, Seq(row, row)), + (s"array($geom2, $geom2)", geometryType2, Seq(row, row)), + (s"array($geom1, $geom2)", mixedSridGeometryType, Seq(row, row)), + (s"array($geom2, $geom1)", mixedSridGeometryType, Seq(row, row)) + ) + + for ((expr, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $expr", + ArrayType(expectedType) + ) + checkAnswer( + sql(s"WITH t AS (SELECT explode($expr) AS g) SELECT $geo FROM t"), + expectedRows + ) + } + } + + test("CreateArray with GEOMETRY columns") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with columns, using geometries with different SRID values. + val geom1 = "ST_GeomFromWKB(wkb)" // Column with fixed SRID (0). + val geometryType1 = GeometryType(0) + val geom2 = s"$geom1::GEOMETRY(ANY)" // Column with mixed SRID (ANY). + val geometryType2 = GeometryType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 0) + + val testCases = Seq( + (s"array($geom1)", geometryType1, Seq(row)), + (s"array($geom2)", geometryType2, Seq(row)), + (s"array($geom1, $geom1)", geometryType1, Seq(row, row)), + (s"array($geom2, $geom2)", geometryType2, Seq(row, row)), + (s"array($geom1, $geom2)", mixedSridGeometryType, Seq(row, row)), + (s"array($geom2, $geom1)", mixedSridGeometryType, Seq(row, row)) + ) + + // Test with literal and column, using geometries with different SRID values. + withTable("tbl") { + // Construct and populate the test table. + sql("CREATE TABLE tbl (wkb BINARY)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString')") + + for ((query, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $query FROM tbl", + ArrayType(expectedType) + ) + checkAnswer( + sql(s"WITH t AS (SELECT explode($query) AS g FROM tbl) SELECT $geo FROM t"), + expectedRows + ) + } + } + } + + test("NVL with GEOMETRY literals") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with literals, using geometries with different SRID values. + val geom1 = s"ST_GeomFromWKB(X'$wkbString')" // Literal with fixed SRID (0). + val geometryType1 = GeometryType(0) + val geom2 = s"$geom1::GEOMETRY(ANY)" // Literal with mixed SRID (ANY). + val geometryType2 = GeometryType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 0) + + val testCases = Seq( + (s"nvl(null, $geom1)", geometryType1, Seq(row)), + (s"nvl($geom1, null)", geometryType1, Seq(row)), + (s"nvl(null, $geom2)", geometryType2, Seq(row)), + (s"nvl($geom2, null)", geometryType2, Seq(row)), + (s"nvl($geom1, $geom1)", geometryType1, Seq(row)), + (s"nvl($geom2, $geom2)", geometryType2, Seq(row)), + (s"nvl($geom1, $geom2)", mixedSridGeometryType, Seq(row)), + (s"nvl($geom2, $geom1)", mixedSridGeometryType, Seq(row)) + ) + + for ((expr, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $expr", + expectedType + ) + checkAnswer( + sql(s"WITH t AS (SELECT $expr AS g) SELECT $geo FROM t"), + expectedRows + ) + } + } + + test("NVL with GEOMETRY columns") { + // Test data: WKB representation of POINT(1 2). + val wkbString = "0101000000000000000000F03F0000000000000040" + // Test with columns, using geometries with different SRID values. + val geom1 = "ST_GeomFromWKB(wkb)" // Column with fixed SRID (0). + val geometryType1 = GeometryType(0) + val geom2 = s"$geom1::GEOMETRY(ANY)" // Column with mixed SRID (ANY). + val geometryType2 = GeometryType("ANY") + val geo = "hex(ST_AsBinary(g)), ST_Srid(g)" + val row = Row(wkbString, 0) + + val testCases = Seq( + (s"nvl(null, $geom1)", geometryType1, Seq(row)), + (s"nvl($geom1, null)", geometryType1, Seq(row)), + (s"nvl(null, $geom2)", geometryType2, Seq(row)), + (s"nvl($geom2, null)", geometryType2, Seq(row)), + (s"nvl($geom1, $geom1)", geometryType1, Seq(row)), + (s"nvl($geom2, $geom2)", geometryType2, Seq(row)), + (s"nvl($geom1, $geom2)", mixedSridGeometryType, Seq(row)), + (s"nvl($geom2, $geom1)", mixedSridGeometryType, Seq(row)) + ) + + // Test with literal and column, using geometries with different SRID values. + withTable("tbl") { + // Construct and populate the test table. + sql("CREATE TABLE tbl (wkb BINARY)") + sql(s"INSERT INTO tbl VALUES (X'$wkbString')") + + for ((query, expectedType, expectedRows) <- testCases) { + assertType( + s"SELECT $query FROM tbl", + expectedType + ) + checkAnswer( + sql(s"WITH t AS (SELECT $query AS g FROM tbl) SELECT $geo FROM t"), + expectedRows + ) + } + } + } + /** ST reader/writer expressions. */ test("ST_AsBinary") { From def1520530cc4d3524506238751df6832dd4e982 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Sat, 8 Nov 2025 07:31:54 -0800 Subject: [PATCH 077/400] [SPARK-53525][CONNECT][FOLLOWUP][4.1] Spark Connect ArrowBatch Result Chunking - Scala Client ### What changes were proposed in this pull request? (This PR is a backporting PR containing https://github.com/apache/spark/pull/52496 and the test fix https://github.com/apache/spark/pull/52941.) In the previous PR https://github.com/apache/spark/pull/52271 of Spark Connect ArrowBatch Result Chunking, both Server-side and PySpark client changes were implemented. In this PR, the corresponding Scala client changes are implemented, so large Arrow rows are now supported on the Scala client as well. To reproduce the existing issue we are solving here, run this code on Spark Connect Scala client: ``` val res = spark.sql("select repeat('a', 1024*1024*300)").collect() println(res(0).getString(0).length) ``` It fails with `RESOURCE_EXHAUSTED` error with message `gRPC message exceeds maximum size 134217728: 314573320`, because the server is trying to send an ExecutePlanResponse of ~300MB to the client. With the improvement introduced by the PR, the above code runs successfully and prints the expected result. ### Why are the changes needed? It improves Spark Connect stability when returning large rows. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52953 from xi-db/arrow-batch-chuking-scala-client@4.1. Authored-by: Xi Lyu Signed-off-by: Dongjoon Hyun --- .../sql/connect/ClientE2ETestSuite.scala | 159 +++++++++++++++++- .../sql/connect/test/RemoteSparkSession.scala | 29 ++-- .../connect/client/SparkConnectClient.scala | 45 ++++- .../sql/connect/client/SparkResult.scala | 138 ++++++++++----- 4 files changed, 313 insertions(+), 58 deletions(-) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala index 8c336b6fa6d5d..450ff8ca62490 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala @@ -26,12 +26,14 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.{DurationInt, FiniteDuration} import scala.jdk.CollectionConverters._ +import io.grpc.{CallOptions, Channel, ClientCall, ClientInterceptor, ForwardingClientCall, ForwardingClientCallListener, MethodDescriptor} import org.apache.commons.io.output.TeeOutputStream import org.scalactic.TolerantNumerics import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkArithmeticException, SparkException, SparkUpgradeException} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} +import org.apache.spark.connect.proto import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.sql.{functions, AnalysisException, Observation, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, TableAlreadyExistsException, TempTableAlreadyExistsException} @@ -41,7 +43,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connect.ConnectConversions._ import org.apache.spark.sql.connect.client.{RetryPolicy, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.test.{ConnectFunSuite, IntegrationTestUtils, QueryTest, RemoteSparkSession, SQLHelper} -import org.apache.spark.sql.connect.test.SparkConnectServerUtils.port +import org.apache.spark.sql.connect.test.SparkConnectServerUtils.{createSparkSession, port} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ @@ -1848,6 +1850,161 @@ class ClientE2ETestSuite checkAnswer(df, Seq.empty) } } + + // Helper class to capture Arrow batch chunk information from gRPC responses + private class ArrowBatchInterceptor extends ClientInterceptor { + case class BatchInfo( + batchIndex: Int, + rowCount: Long, + startOffset: Long, + chunks: Seq[ChunkInfo]) { + def totalChunks: Int = chunks.length + } + + case class ChunkInfo( + batchIndex: Int, + chunkIndex: Int, + numChunksInBatch: Int, + rowCount: Long, + startOffset: Long, + dataSize: Int) + + private val batches: mutable.Buffer[BatchInfo] = mutable.Buffer.empty + private var currentBatchIndex: Int = 0 + private val currentBatchChunks: mutable.Buffer[ChunkInfo] = mutable.Buffer.empty + + override def interceptCall[ReqT, RespT]( + method: MethodDescriptor[ReqT, RespT], + callOptions: CallOptions, + next: Channel): ClientCall[ReqT, RespT] = { + new ForwardingClientCall.SimpleForwardingClientCall[ReqT, RespT]( + next.newCall(method, callOptions)) { + override def start( + responseListener: ClientCall.Listener[RespT], + headers: io.grpc.Metadata): Unit = { + super.start( + new ForwardingClientCallListener.SimpleForwardingClientCallListener[RespT]( + responseListener) { + override def onMessage(message: RespT): Unit = { + message match { + case response: proto.ExecutePlanResponse if response.hasArrowBatch => + val arrowBatch = response.getArrowBatch + // Track chunk information for every chunk + currentBatchChunks += ChunkInfo( + batchIndex = currentBatchIndex, + chunkIndex = arrowBatch.getChunkIndex.toInt, + numChunksInBatch = arrowBatch.getNumChunksInBatch.toInt, + rowCount = arrowBatch.getRowCount, + startOffset = arrowBatch.getStartOffset, + dataSize = arrowBatch.getData.size()) + // When we receive the last chunk, create the BatchInfo + if (currentBatchChunks.length == arrowBatch.getNumChunksInBatch) { + batches += BatchInfo( + batchIndex = currentBatchIndex, + rowCount = arrowBatch.getRowCount, + startOffset = arrowBatch.getStartOffset, + chunks = currentBatchChunks.toList) + currentBatchChunks.clear() + currentBatchIndex += 1 + } + case _ => // Not an ExecutePlanResponse with ArrowBatch, ignore + } + super.onMessage(message) + } + }, + headers) + } + } + } + + // Get all batch information + def getBatchInfos: Seq[BatchInfo] = batches.toSeq + + def clear(): Unit = { + currentBatchIndex = 0 + currentBatchChunks.clear() + batches.clear() + } + } + + test("Arrow batch result chunking") { + // This test validates that the client can correctly reassemble chunked Arrow batches + // using SequenceInputStream as implemented in SparkResult.processResponses + + // Two cases are tested here: + // (a) client preferred chunk size is set: the server should respect it + // (b) client preferred chunk size is not set: the server should use its own max chunk size + Seq((Some(1024), None), (None, Some(1024))).foreach { + case (preferredChunkSizeOpt, maxChunkSizeOpt) => + // Create interceptor to capture chunk information + val arrowBatchInterceptor = new ArrowBatchInterceptor() + + try { + // Set preferred chunk size if specified and add interceptor + preferredChunkSizeOpt match { + case Some(size) => + spark = createSparkSession( + _.preferredArrowChunkSize(Some(size)).interceptor(arrowBatchInterceptor)) + case None => + spark = createSparkSession(_.interceptor(arrowBatchInterceptor)) + } + // Set server max chunk size if specified + maxChunkSizeOpt.foreach { size => + spark.conf.set("spark.connect.session.resultChunking.maxChunkSize", size.toString) + } + + val sqlQuery = + "select id, CAST(id + 0.5 AS DOUBLE) as double_val from range(0, 2000, 1, 4)" + + // Execute the query using withResult to access SparkResult object + spark.sql(sqlQuery).withResult { result => + // Verify the results are correct and complete + assert(result.length == 2000) + + // Get batch information from interceptor + val batchInfos = arrowBatchInterceptor.getBatchInfos + + // Assert there are 4 batches (partitions) in total + assert(batchInfos.length == 4) + + // Validate chunk information for each batch + val maxChunkSize = preferredChunkSizeOpt.orElse(maxChunkSizeOpt).get + batchInfos.foreach { batch => + // In this example, the max chunk size is set to a small value, + // so each Arrow batch should be split into multiple chunks + assert(batch.totalChunks > 5) + assert(batch.chunks.nonEmpty) + assert(batch.chunks.length == batch.totalChunks) + batch.chunks.zipWithIndex.foreach { case (chunk, expectedIndex) => + assert(chunk.chunkIndex == expectedIndex) + assert(chunk.numChunksInBatch == batch.totalChunks) + assert(chunk.rowCount == batch.rowCount) + assert(chunk.startOffset == batch.startOffset) + assert(chunk.dataSize > 0) + assert(chunk.dataSize <= maxChunkSize) + } + } + + // Validate data integrity across the range to ensure chunking didn't corrupt anything + val rows = result.toArray + var expectedId = 0L + rows.foreach { row => + assert(row.getLong(0) == expectedId) + val expectedDouble = expectedId + 0.5 + val actualDouble = row.getDouble(1) + assert(math.abs(actualDouble - expectedDouble) < 0.001) + expectedId += 1 + } + } + } finally { + // Clean up configurations + maxChunkSizeOpt.foreach { _ => + spark.conf.unset("spark.connect.session.resultChunking.maxChunkSize") + } + arrowBatchInterceptor.clear() + } + } + } } private[sql] case class ClassData(a: String, b: Int) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala index efb6c721876c3..6d8d2edcf0821 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala @@ -180,26 +180,35 @@ object SparkConnectServerUtils { val fileName = e.substring(e.lastIndexOf(File.separatorChar) + 1) fileName.endsWith(".jar") && (fileName.startsWith("scalatest") || fileName.startsWith("scalactic") || - (fileName.startsWith("spark-catalyst") && fileName.endsWith("-tests"))) + (fileName.startsWith("spark-catalyst") && fileName.endsWith("-tests")) || + fileName.startsWith("grpc-")) } .map(e => Paths.get(e).toUri) spark.client.artifactManager.addArtifacts(jars.toImmutableArraySeq) } def createSparkSession(): SparkSession = { + createSparkSession(identity) + } + + def createSparkSession( + customBuilderFunc: SparkConnectClient.Builder => SparkConnectClient.Builder) + : SparkSession = { SparkConnectServerUtils.start() + var builder = SparkConnectClient + .builder() + .userId("test") + .port(port) + .retryPolicy( + RetryPolicy + .defaultPolicy() + .copy(maxRetries = Some(10), maxBackoff = Some(FiniteDuration(30, "s")))) + + builder = customBuilderFunc(builder) val spark = SparkSession .builder() - .client( - SparkConnectClient - .builder() - .userId("test") - .port(port) - .retryPolicy(RetryPolicy - .defaultPolicy() - .copy(maxRetries = Some(10), maxBackoff = Some(FiniteDuration(30, "s")))) - .build()) + .client(builder.build()) .create() // Execute an RPC which will get retried until the server is up. diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index fa32eba91eb2c..e5fd16a7c2612 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -138,6 +138,22 @@ private[sql] class SparkConnectClient( .setSessionId(sessionId) .setClientType(userAgent) .addAllTags(tags.get.toSeq.asJava) + + // Add request option to allow result chunking. + if (configuration.allowArrowBatchChunking) { + val chunkingOptionsBuilder = proto.ResultChunkingOptions + .newBuilder() + .setAllowArrowBatchChunking(true) + configuration.preferredArrowChunkSize.foreach { size => + chunkingOptionsBuilder.setPreferredArrowChunkSize(size) + } + request.addRequestOptions( + proto.ExecutePlanRequest.RequestOption + .newBuilder() + .setResultChunkingOptions(chunkingOptionsBuilder.build()) + .build()) + } + serverSideSessionId.foreach(session => request.setClientObservedServerSideSessionId(session)) operationId.foreach { opId => require( @@ -332,6 +348,16 @@ private[sql] class SparkConnectClient( def copy(): SparkConnectClient = configuration.toSparkConnectClient + /** + * Returns whether arrow batch chunking is allowed. + */ + def allowArrowBatchChunking: Boolean = configuration.allowArrowBatchChunking + + /** + * Returns the preferred arrow chunk size in bytes. + */ + def preferredArrowChunkSize: Option[Int] = configuration.preferredArrowChunkSize + /** * Add a single artifact to the client session. * @@ -757,6 +783,21 @@ object SparkConnectClient { this } + def allowArrowBatchChunking(allow: Boolean): Builder = { + _configuration = _configuration.copy(allowArrowBatchChunking = allow) + this + } + + def allowArrowBatchChunking: Boolean = _configuration.allowArrowBatchChunking + + def preferredArrowChunkSize(size: Option[Int]): Builder = { + size.foreach(s => require(s > 0, "preferredArrowChunkSize must be positive")) + _configuration = _configuration.copy(preferredArrowChunkSize = size) + this + } + + def preferredArrowChunkSize: Option[Int] = _configuration.preferredArrowChunkSize + def build(): SparkConnectClient = _configuration.toSparkConnectClient } @@ -801,7 +842,9 @@ object SparkConnectClient { interceptors: List[ClientInterceptor] = List.empty, sessionId: Option[String] = None, grpcMaxMessageSize: Int = ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE, - grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) { + grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT, + allowArrowBatchChunking: Boolean = true, + preferredArrowChunkSize: Option[Int] = None) { private def isLocal = host.equals("localhost") diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index ef55edd10c8a3..43265e55a0ca9 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -16,18 +16,21 @@ */ package org.apache.spark.sql.connect.client +import java.io.SequenceInputStream import java.lang.ref.Cleaner import java.util.Objects import scala.collection.mutable import scala.jdk.CollectionConverters._ +import com.google.protobuf.ByteString import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch} import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics +import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} @@ -42,7 +45,8 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String) - extends AutoCloseable { self => + extends AutoCloseable + with Logging { self => case class StageInfo( stageId: Long, @@ -118,6 +122,7 @@ private[sql] class SparkResult[T]( stopOnFirstNonEmptyResponse: Boolean = false): Boolean = { var nonEmpty = false var stop = false + val arrowBatchChunksToAssemble = mutable.Buffer.empty[ByteString] while (!stop && responses.hasNext) { val response = responses.next() @@ -151,55 +156,96 @@ private[sql] class SparkResult[T]( stop |= stopOnSchema } if (response.hasArrowBatch) { - val ipcStreamBytes = response.getArrowBatch.getData - val expectedNumRows = response.getArrowBatch.getRowCount - val reader = new MessageIterator(ipcStreamBytes.newInput(), allocator) - if (arrowSchema == null) { - arrowSchema = reader.schema - stop |= stopOnArrowSchema - } else if (arrowSchema != reader.schema) { - throw new IllegalStateException( - s"""Schema Mismatch between expected and received schema: - |=== Expected Schema === - |$arrowSchema - |=== Received Schema === - |${reader.schema} - |""".stripMargin) - } - if (structType == null) { - // If the schema is not available yet, fallback to the arrow schema. - structType = ArrowUtils.fromArrowSchema(reader.schema) - } - if (response.getArrowBatch.hasStartOffset) { - val expectedStartOffset = response.getArrowBatch.getStartOffset - if (numRecords != expectedStartOffset) { + val arrowBatch = response.getArrowBatch + logDebug( + s"Received arrow batch rows=${arrowBatch.getRowCount} " + + s"Number of chunks in batch=${arrowBatch.getNumChunksInBatch} " + + s"Chunk index=${arrowBatch.getChunkIndex} " + + s"size=${arrowBatch.getData.size()}") + + if (arrowBatchChunksToAssemble.nonEmpty) { + // Expect next chunk of the same batch + if (arrowBatch.getChunkIndex != arrowBatchChunksToAssemble.size) { throw new IllegalStateException( - s"Expected arrow batch to start at row offset $numRecords in results, " + - s"but received arrow batch starting at offset $expectedStartOffset.") + s"Expected chunk index ${arrowBatchChunksToAssemble.size} of the " + + s"arrow batch but got ${arrowBatch.getChunkIndex}.") } - } - var numRecordsInBatch = 0 - val messages = Seq.newBuilder[ArrowMessage] - while (reader.hasNext) { - val message = reader.next() - message match { - case batch: ArrowRecordBatch => - numRecordsInBatch += batch.getLength - case _ => + } else { + // Expect next batch + if (arrowBatch.hasStartOffset) { + val expectedStartOffset = arrowBatch.getStartOffset + if (numRecords != expectedStartOffset) { + throw new IllegalStateException( + s"Expected arrow batch to start at row offset $numRecords in results, " + + s"but received arrow batch starting at offset $expectedStartOffset.") + } + } + if (arrowBatch.getChunkIndex != 0) { + throw new IllegalStateException( + s"Expected chunk index 0 of the next arrow batch " + + s"but got ${arrowBatch.getChunkIndex}.") } - messages += message - } - if (numRecordsInBatch != expectedNumRows) { - throw new IllegalStateException( - s"Expected $expectedNumRows rows in arrow batch but got $numRecordsInBatch.") } - // Skip the entire result if it is empty. - if (numRecordsInBatch > 0) { - numRecords += numRecordsInBatch - resultMap.put(nextResultIndex, (reader.bytesRead, messages.result())) - nextResultIndex += 1 - nonEmpty |= true - stop |= stopOnFirstNonEmptyResponse + + arrowBatchChunksToAssemble += arrowBatch.getData + + // Assemble the chunks to an arrow batch to process if + // (a) chunking is not enabled (numChunksInBatch is not set or is 0, + // in this case, it is the single chunk in the batch) + // (b) or the client has received all chunks of the batch. + if (!arrowBatch.hasNumChunksInBatch || + arrowBatch.getNumChunksInBatch == 0 || + arrowBatchChunksToAssemble.size == arrowBatch.getNumChunksInBatch) { + + val numChunks = arrowBatchChunksToAssemble.size + val inputStreams = + arrowBatchChunksToAssemble.map(_.newInput()).iterator.asJavaEnumeration + val input = new SequenceInputStream(inputStreams) + arrowBatchChunksToAssemble.clear() + logDebug(s"Assembling arrow batch from $numChunks chunks.") + + val expectedNumRows = arrowBatch.getRowCount + val reader = new MessageIterator(input, allocator) + if (arrowSchema == null) { + arrowSchema = reader.schema + stop |= stopOnArrowSchema + } else if (arrowSchema != reader.schema) { + throw new IllegalStateException( + s"""Schema Mismatch between expected and received schema: + |=== Expected Schema === + |$arrowSchema + |=== Received Schema === + |${reader.schema} + |""".stripMargin) + } + if (structType == null) { + // If the schema is not available yet, fallback to the arrow schema. + structType = ArrowUtils.fromArrowSchema(reader.schema) + } + + var numRecordsInBatch = 0 + val messages = Seq.newBuilder[ArrowMessage] + while (reader.hasNext) { + val message = reader.next() + message match { + case batch: ArrowRecordBatch => + numRecordsInBatch += batch.getLength + case _ => + } + messages += message + } + if (numRecordsInBatch != expectedNumRows) { + throw new IllegalStateException( + s"Expected $expectedNumRows rows in arrow batch but got $numRecordsInBatch.") + } + // Skip the entire result if it is empty. + if (numRecordsInBatch > 0) { + numRecords += numRecordsInBatch + resultMap.put(nextResultIndex, (reader.bytesRead, messages.result())) + nextResultIndex += 1 + nonEmpty |= true + stop |= stopOnFirstNonEmptyResponse + } } } } From 3ccc769642ff216f1301681de400fedb2d774073 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Sat, 8 Nov 2025 07:35:47 -0800 Subject: [PATCH 078/400] [SPARK-54166][GEO][PYTHON] Introduce type encoders for geospatial types in PySpark ### What changes were proposed in this pull request? This PR introduces type encoders for `Geography` and `Geometry` in PySpark. Note that Scala-side encoders for geospatial types were added in: https://github.com/apache/spark/pull/52813. ### Why are the changes needed? Expanding client support for geospatial types in PySpark API. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New PySpark unit tests: - `test_types` - `test_parity_types` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52861 from uros-db/geo-pyspark-encoders. Authored-by: Uros Bojanic Signed-off-by: Dongjoon Hyun (cherry picked from commit d304a0313ff14948c24506500f661f415f16f9bc) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/__init__.py | 4 +- python/pyspark/sql/pandas/types.py | 120 +++++++ .../sql/tests/connect/test_parity_types.py | 4 + python/pyspark/sql/tests/test_types.py | 322 ++++++++++++++++++ python/pyspark/sql/types.py | 32 ++ .../sql/execution/python/EvaluatePython.scala | 27 +- 6 files changed, 505 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index a0a6e8ef70c8d..eeeeddd00e3af 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -39,7 +39,7 @@ - :class:`pyspark.sql.Window` For working with window functions. """ -from pyspark.sql.types import Row, VariantVal +from pyspark.sql.types import Geography, Geometry, Row, VariantVal from pyspark.sql.context import SQLContext, HiveContext, UDFRegistration, UDTFRegistration from pyspark.sql.session import SparkSession from pyspark.sql.column import Column @@ -69,6 +69,8 @@ "DataFrameNaFunctions", "DataFrameStatFunctions", "VariantVal", + "Geography", + "Geometry", "Window", "WindowSpec", "DataFrameReader", diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index 327e3941d9386..d8a45daa77e89 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -50,6 +50,10 @@ UserDefinedType, VariantType, VariantVal, + GeometryType, + Geometry, + GeographyType, + Geography, _create_row, ) from pyspark.errors import PySparkTypeError, UnsupportedOperationException, PySparkValueError @@ -202,6 +206,28 @@ def to_arrow_type( pa.field("metadata", pa.binary(), nullable=False, metadata={b"variant": b"true"}), ] arrow_type = pa.struct(fields) + elif type(dt) == GeometryType: + fields = [ + pa.field("srid", pa.int32(), nullable=False), + pa.field( + "wkb", + pa.binary(), + nullable=False, + metadata={b"geometry": b"true", b"srid": str(dt.srid)}, + ), + ] + arrow_type = pa.struct(fields) + elif type(dt) == GeographyType: + fields = [ + pa.field("srid", pa.int32(), nullable=False), + pa.field( + "wkb", + pa.binary(), + nullable=False, + metadata={b"geography": b"true", b"srid": str(dt.srid)}, + ), + ] + arrow_type = pa.struct(fields) else: raise PySparkTypeError( errorClass="UNSUPPORTED_DATA_TYPE_FOR_ARROW_CONVERSION", @@ -272,6 +298,38 @@ def is_variant(at: "pa.DataType") -> bool: ) and any(field.name == "value" for field in at) +def is_geometry(at: "pa.DataType") -> bool: + """Check if a PyArrow struct data type represents a geometry""" + import pyarrow.types as types + + assert types.is_struct(at) + + return any( + ( + field.name == "wkb" + and b"geometry" in field.metadata + and field.metadata[b"geometry"] == b"true" + ) + for field in at + ) and any(field.name == "srid" for field in at) + + +def is_geography(at: "pa.DataType") -> bool: + """Check if a PyArrow struct data type represents a geography""" + import pyarrow.types as types + + assert types.is_struct(at) + + return any( + ( + field.name == "wkb" + and b"geography" in field.metadata + and field.metadata[b"geography"] == b"true" + ) + for field in at + ) and any(field.name == "srid" for field in at) + + def from_arrow_type(at: "pa.DataType", prefer_timestamp_ntz: bool = False) -> DataType: """Convert pyarrow type to Spark data type.""" import pyarrow.types as types @@ -337,6 +395,18 @@ def from_arrow_type(at: "pa.DataType", prefer_timestamp_ntz: bool = False) -> Da elif types.is_struct(at): if is_variant(at): return VariantType() + elif is_geometry(at): + srid = int(at.field("wkb").metadata.get(b"srid")) + if srid == GeometryType.MIXED_SRID: + return GeometryType("ANY") + else: + return GeometryType(srid) + elif is_geography(at): + srid = int(at.field("wkb").metadata.get(b"srid")) + if srid == GeographyType.MIXED_SRID: + return GeographyType("ANY") + else: + return GeographyType(srid) return StructType( [ StructField( @@ -1098,6 +1168,40 @@ def convert_variant(value: Any) -> Any: return convert_variant + elif isinstance(dt, GeographyType): + + def convert_geography(value: Any) -> Any: + if value is None: + return None + elif ( + isinstance(value, dict) + and all(key in value for key in ["wkb", "srid"]) + and isinstance(value["wkb"], bytes) + and isinstance(value["srid"], int) + ): + return Geography.fromWKB(value["wkb"], value["srid"]) + else: + raise PySparkValueError(errorClass="MALFORMED_GEOGRAPHY") + + return convert_geography + + elif isinstance(dt, GeometryType): + + def convert_geometry(value: Any) -> Any: + if value is None: + return None + elif ( + isinstance(value, dict) + and all(key in value for key in ["wkb", "srid"]) + and isinstance(value["wkb"], bytes) + and isinstance(value["srid"], int) + ): + return Geometry.fromWKB(value["wkb"], value["srid"]) + else: + raise PySparkValueError(errorClass="MALFORMED_GEOMETRY") + + return convert_geometry + else: return None @@ -1360,6 +1464,22 @@ def convert_variant(variant: Any) -> Any: return convert_variant + elif isinstance(dt, GeographyType): + + def convert_geography(value: Any) -> Any: + assert isinstance(value, Geography) + return {"srid": value.srid, "wkb": value.wkb} + + return convert_geography + + elif isinstance(dt, GeometryType): + + def convert_geometry(value: Any) -> Any: + assert isinstance(value, Geometry) + return {"srid": value.srid, "wkb": value.wkb} + + return convert_geometry + return None conv = _converter(data_type) diff --git a/python/pyspark/sql/tests/connect/test_parity_types.py b/python/pyspark/sql/tests/connect/test_parity_types.py index 6d06611def6af..a39e92233bc0e 100644 --- a/python/pyspark/sql/tests/connect/test_parity_types.py +++ b/python/pyspark/sql/tests/connect/test_parity_types.py @@ -34,6 +34,10 @@ def test_apply_schema_to_dict_and_rows(self): def test_apply_schema_to_row(self): super().test_apply_schema_to_row() + @unittest.skip("Spark Connect does not support RDD but the tests depend on them.") + def test_geospatial_create_dataframe_rdd(self): + super().test_geospatial_create_dataframe_rdd() + @unittest.skip("Spark Connect does not support RDD but the tests depend on them.") def test_create_dataframe_schema_mismatch(self): super().test_create_dataframe_schema_mismatch() diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 6979095acca88..4ff2ab3e5cd73 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -29,6 +29,8 @@ from pyspark.sql import functions as F from pyspark.errors import ( AnalysisException, + IllegalArgumentException, + SparkRuntimeException, ParseException, PySparkTypeError, PySparkValueError, @@ -51,6 +53,8 @@ MapType, StringType, CharType, + Geography, + Geometry, VarcharType, StructType, StructField, @@ -1365,6 +1369,7 @@ def test_parse_datatype_json_string(self): NullType(), GeographyType(4326), GeographyType("ANY"), + GeometryType(0), GeometryType(4326), GeometryType("ANY"), VariantType(), @@ -2447,6 +2452,323 @@ def test_variant_type(self): with self.assertRaises(PySparkValueError, msg="Rows cannot be of type VariantVal"): self.spark.createDataFrame([VariantVal.parseJson("2")], "v variant") + def test_geospatial_encoding(self): + df = self.spark.createDataFrame( + [ + ( + bytes.fromhex("0101000000000000000000F03F0000000000000040"), + 4326, + ) + ], + ["wkb", "srid"], + ) + row = df.select( + F.st_geomfromwkb(df.wkb).alias("geom"), + F.st_geogfromwkb(df.wkb).alias("geog"), + ).collect()[0] + + self.assertEqual(type(row["geom"]), Geometry) + self.assertEqual(type(row["geog"]), Geography) + self.assertEqual( + row["geom"].getBytes(), bytes.fromhex("0101000000000000000000F03F0000000000000040") + ) + self.assertEqual(row["geom"].getSrid(), 0) + self.assertEqual( + row["geog"].getBytes(), bytes.fromhex("0101000000000000000000F03F0000000000000040") + ) + self.assertEqual(row["geog"].getSrid(), 4326) + + def test_geospatial_create_dataframe_rdd(self): + schema = StructType( + [ + StructField("id", IntegerType(), True), + StructField("geom", GeometryType(0), True), + StructField("geom4326", GeometryType(4326), True), + StructField("geog", GeographyType(4326), True), + ] + ) + geospatial_data = [ + ( + 1, + Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 0), + Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 4326), + Geography.fromWKB( + bytes.fromhex("010100000000000000000031400000000000001c40"), 4326 + ), + ), + ( + 2, + Geometry.fromWKB(bytes.fromhex("010100000000000000000014400000000000001440"), 0), + Geometry.fromWKB(bytes.fromhex("010100000000000000000014400000000000001440"), 4326), + Geography.fromWKB( + bytes.fromhex("010100000000000000000014400000000000001440"), 4326 + ), + ), + ] + rdd_data = self.sc.parallelize(geospatial_data) + df = self.spark.createDataFrame(rdd_data, schema) + rows = df.select( + F.st_asbinary(df.geom).alias("geom_wkb"), + F.st_srid(df.geom).alias("geom_srid"), + F.st_asbinary(df.geom4326).alias("geom4326_wkb"), + F.st_srid(df.geom4326).alias("geom4326_srid"), + F.st_asbinary(df.geog).alias("geog_wkb"), + F.st_srid(df.geog).alias("geog_srid"), + ).collect() + + point0_wkb = bytes.fromhex("010100000000000000000031400000000000001c40") + point1_wkb = bytes.fromhex("010100000000000000000014400000000000001440") + self.assertEqual(rows[0]["geom_wkb"], point0_wkb) + self.assertEqual(rows[0]["geom4326_wkb"], point0_wkb) + self.assertEqual(rows[0]["geog_wkb"], point0_wkb) + self.assertEqual(rows[1]["geom_wkb"], point1_wkb) + self.assertEqual(rows[1]["geom4326_wkb"], point1_wkb) + self.assertEqual(rows[1]["geog_wkb"], point1_wkb) + self.assertEqual(rows[0]["geom_srid"], 0) + self.assertEqual(rows[0]["geom4326_srid"], 4326) + self.assertEqual(rows[0]["geog_srid"], 4326) + self.assertEqual(rows[1]["geom_srid"], 0) + self.assertEqual(rows[1]["geom4326_srid"], 4326) + self.assertEqual(rows[1]["geog_srid"], 4326) + schema_df = self.spark.createDataFrame(rdd_data).select( + F.col("_1").alias("id"), + F.col("_2").alias("geom"), + F.col("_3").alias("geom4326"), + F.col("_4").alias("geog"), + ) + self.assertEqual(df.collect(), schema_df.collect()) + + def test_geospatial_create_dataframe(self): + # Positive Test: Creating DataFrame from a list of tuples with explicit schema + geospatial_data = [ + ( + 1, + Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 0), + Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 4326), + Geography.fromWKB( + bytes.fromhex("010100000000000000000031400000000000001c40"), 4326 + ), + ), + ( + 2, + Geometry.fromWKB(bytes.fromhex("010100000000000000000014400000000000001440"), 0), + Geometry.fromWKB(bytes.fromhex("010100000000000000000014400000000000001440"), 4326), + Geography.fromWKB( + bytes.fromhex("010100000000000000000014400000000000001440"), 4326 + ), + ), + ] + named_geospatial_data = [ + { + "id": 1, + "geom": Geometry.fromWKB( + bytes.fromhex("010100000000000000000031400000000000001c40"), 0 + ), + "geom4326": Geometry.fromWKB( + bytes.fromhex("010100000000000000000031400000000000001c40"), 4326 + ), + "geog": Geography.fromWKB( + bytes.fromhex("010100000000000000000031400000000000001c40"), 4326 + ), + }, + { + "id": 2, + "geom": Geometry.fromWKB( + bytes.fromhex("010100000000000000000014400000000000001440"), 0 + ), + "geom4326": Geometry.fromWKB( + bytes.fromhex("010100000000000000000014400000000000001440"), 4326 + ), + "geog": Geography.fromWKB( + bytes.fromhex("010100000000000000000014400000000000001440"), 4326 + ), + }, + ] + GeospatialRow = Row("id", "geom", "geom4326", "geog") + spark_row_data = [ + GeospatialRow( + 1, + Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 0), + Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 4326), + Geography.fromWKB( + bytes.fromhex("010100000000000000000031400000000000001c40"), 4326 + ), + ), + GeospatialRow( + 2, + Geometry.fromWKB(bytes.fromhex("010100000000000000000014400000000000001440"), 0), + Geometry.fromWKB(bytes.fromhex("010100000000000000000014400000000000001440"), 4326), + Geography.fromWKB( + bytes.fromhex("010100000000000000000014400000000000001440"), 4326 + ), + ), + ] + schema = StructType( + [ + StructField("id", IntegerType(), True), + StructField("geom", GeometryType(0), True), + StructField("geom4326", GeometryType(4326), True), + StructField("geog", GeographyType(4326), True), + ] + ) + # Negative Test: Schema mismatch + mismatched_schema = StructType( + [ + StructField("id", IntegerType(), True), # Should be GeometryType + StructField("geom", GeometryType(4326), True), # Should be GeometryType + StructField("geom4326", GeometryType(4326), True), # Should be GeometryType + StructField("geog", GeographyType(4326), True), # Should be GeographyType + ] + ) + + # Explicitly validate single test case + # rest will be compared with this one. + df = self.spark.createDataFrame(geospatial_data, schema) + rows = df.select( + F.st_asbinary(df.geom).alias("geom_wkb"), + F.st_srid(df.geom).alias("geom_srid"), + F.st_asbinary(df.geom4326).alias("geom4326_wkb"), + F.st_srid(df.geom4326).alias("geom4326_srid"), + F.st_asbinary(df.geog).alias("geog_wkb"), + F.st_srid(df.geog).alias("geog_srid"), + ).collect() + + point0_wkb = bytes.fromhex("010100000000000000000031400000000000001c40") + point1_wkb = bytes.fromhex("010100000000000000000014400000000000001440") + self.assertEqual(rows[0]["geom_wkb"], point0_wkb) + self.assertEqual(rows[0]["geom4326_wkb"], point0_wkb) + self.assertEqual(rows[0]["geog_wkb"], point0_wkb) + self.assertEqual(rows[1]["geom_wkb"], point1_wkb) + self.assertEqual(rows[1]["geom4326_wkb"], point1_wkb) + self.assertEqual(rows[1]["geog_wkb"], point1_wkb) + self.assertEqual(rows[0]["geom_srid"], 0) + self.assertEqual(rows[0]["geom4326_srid"], 4326) + self.assertEqual(rows[0]["geog_srid"], 4326) + self.assertEqual(rows[1]["geom_srid"], 0) + self.assertEqual(rows[1]["geom4326_srid"], 4326) + self.assertEqual(rows[1]["geog_srid"], 4326) + + # Just the data set without parameters. + self.assertEqual( + self.spark.createDataFrame(named_geospatial_data) + .select("id", "geom", "geom4326", "geog") + .collect(), + df.collect(), + ) + self.assertEqual(self.spark.createDataFrame(geospatial_data).collect(), df.collect()) + self.assertEqual(self.spark.createDataFrame(spark_row_data).collect(), df.collect()) + + # Define DataFrame creation methods + datasets = [named_geospatial_data, geospatial_data, spark_row_data] + schemas = [ + schema, + "id INT, geom GEOMETRY(0), geom4326 GEOMETRY(4326), geog GEOGRAPHY(4326)", + ["id", "geom", "geom4326", "geog"], + ] + + for dataset_to_check, schema_to_check in zip(datasets, schemas): + df_to_check = self.spark.createDataFrame(dataset_to_check, schema_to_check).select( + "id", "geom", "geom4326", "geog" + ) + self.assertEqual(df_to_check.collect(), df.collect(), "DataFrame creation with schema") + + # Negative Test: Schema mismatch + for dataset_to_check in datasets: + with self.assertRaises(SparkRuntimeException) as pe: + self.spark.createDataFrame(dataset_to_check, mismatched_schema).collect() + + self.check_error( + exception=pe.exception, + errorClass="GEO_ENCODER_SRID_MISMATCH_ERROR", + messageParameters={"type": "GEOMETRY", "typeSrid": "4326", "valueSrid": "0"}, + ) + + def test_geospatial_schema_inferrence(self): + # Mixed data with different SRIDs + wkb = bytes.fromhex("010100000000000000000031400000000000001c40") + geometry_dataset = [ + (Geometry.fromWKB(wkb, 0), Geometry.fromWKB(wkb, 4326), Geometry.fromWKB(wkb, 4326)), + (Geometry.fromWKB(wkb, 0), Geometry.fromWKB(wkb, 4326), Geometry.fromWKB(wkb, 0)), + (Geometry.fromWKB(wkb, 0), Geometry.fromWKB(wkb, 4326), Geometry.fromWKB(wkb, 4326)), + (Geometry.fromWKB(wkb, 0), Geometry.fromWKB(wkb, 4326), Geometry.fromWKB(wkb, 0)), + ] + # Create DataFrame with mixed data types + df = self.spark.createDataFrame(geometry_dataset, ["geom0", "geom4326", "geom_any"]) + expected_schema = StructType( + [ + StructField("geom0", GeometryType(0), True), + StructField("geom4326", GeometryType(4326), True), + StructField("geom_any", GeometryType("ANY"), True), + ] + ) + self.assertEqual(df.schema, expected_schema) + + rows = df.select( + F.st_asbinary("geom0").alias("geom0_wkb"), + F.st_srid("geom0").alias("geom0_srid"), + F.st_asbinary("geom4326").alias("geom4326_wkb"), + F.st_srid("geom4326").alias("geom4326_srid"), + F.st_asbinary("geom_any").alias("geom_any_wkb"), + F.st_srid("geom_any").alias("geom_any_srid"), + ).collect() + + point_wkb = bytes.fromhex("010100000000000000000031400000000000001c40") + self.assertEqual(rows[0]["geom0_wkb"], point_wkb) + self.assertEqual(rows[1]["geom0_wkb"], point_wkb) + self.assertEqual(rows[0]["geom4326_wkb"], point_wkb) + self.assertEqual(rows[1]["geom4326_wkb"], point_wkb) + self.assertEqual(rows[0]["geom_any_wkb"], point_wkb) + self.assertEqual(rows[1]["geom_any_wkb"], point_wkb) + self.assertEqual(rows[0]["geom0_srid"], 0) + self.assertEqual(rows[1]["geom0_srid"], 0) + self.assertEqual(rows[0]["geom4326_srid"], 4326) + self.assertEqual(rows[1]["geom4326_srid"], 4326) + self.assertEqual(rows[0]["geom_any_srid"], 4326) + self.assertEqual(rows[1]["geom_any_srid"], 0) + + def test_geospatial_mixed_check_srid_validity(self): + geometry_mixed_invalid_data = [ + (1, Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 0)), + (2, Geometry.fromWKB(bytes.fromhex("010100000000000000000031400000000000001c40"), 1)), + ] + + with self.assertRaises(IllegalArgumentException) as pe: + self.spark.createDataFrame(geometry_mixed_invalid_data).collect() + self.check_error( + exception=pe.exception, + errorClass="ST_INVALID_SRID_VALUE", + messageParameters={"srid": "1"}, + ) + + with self.assertRaises(IllegalArgumentException) as pe: + self.spark.createDataFrame( + geometry_mixed_invalid_data, "id INT, geom GEOMETRY(ANY)" + ).collect() + self.check_error( + exception=pe.exception, + errorClass="ST_INVALID_SRID_VALUE", + messageParameters={"srid": "1"}, + ) + + def test_geospatial_result_encoding(self): + point_wkb = "010100000000000000000031400000000000001c40" + point_bytes = bytes.fromhex(point_wkb) + df = self.spark.sql( + f""" + SELECT ST_GeomFromWKB(X'{point_wkb}') AS geom, + ST_GeogFromWKB(X'{point_wkb}') AS geog""" + ) + GeospatialRow = Row("geom", "geog") + self.assertEqual( + df.collect(), + [ + GeospatialRow( + Geometry.fromWKB(point_bytes, 0), + Geography.fromWKB(point_bytes, 4326), + ) + ], + ) + def test_to_ddl(self): schema = StructType().add("a", NullType()).add("b", BooleanType()).add("c", BinaryType()) self.assertEqual(schema.toDDL(), "a VOID,b BOOLEAN,c BINARY") diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 8aae398800727..95307ea3859c7 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -2517,6 +2517,8 @@ def _assert_valid_collation_provider(provider: str) -> None: # Mapping Python types to Spark SQL DataType _type_mappings = { type(None): NullType, + Geometry: GeometryType, + Geography: GeographyType, bool: BooleanType, int: LongType, float: DoubleType, @@ -2648,6 +2650,12 @@ def _infer_type( return obj.__UDT__ dataType = _type_mappings.get(type(obj)) + if dataType is GeographyType: + assert isinstance(obj, Geography) + return GeographyType(obj.getSrid()) + if dataType is GeometryType: + assert isinstance(obj, Geometry) + return GeometryType(obj.getSrid()) if dataType is DecimalType: # the precision and scale of `obj` may be different from row to row. return DecimalType(38, 18) @@ -2915,6 +2923,10 @@ def new_name(n: str) -> str: return a elif isinstance(a, TimestampNTZType) and isinstance(b, TimestampType): return b + elif isinstance(a, GeometryType) and isinstance(b, GeometryType) and a.srid != b.srid: + return GeometryType("ANY") + elif isinstance(a, GeographyType) and isinstance(b, GeographyType) and a.srid != b.srid: + return GeographyType("ANY") elif isinstance(a, AtomicType) and isinstance(b, StringType): return b elif isinstance(a, StringType) and isinstance(b, AtomicType): @@ -3068,6 +3080,8 @@ def convert_struct(obj: Any) -> Optional[Tuple]: ArrayType: (list, tuple, array), MapType: (dict,), StructType: (tuple, list, dict), + GeometryType: (Geometry,), + GeographyType: (Geography,), VariantType: ( bool, int, @@ -3419,6 +3433,24 @@ def verify_variant(obj: Any) -> None: verify_value = verify_variant + elif isinstance(dataType, GeometryType): + + def verify_geometry(obj: Any) -> None: + assert_acceptable_types(obj) + verify_acceptable_types(obj) + assert isinstance(obj, Geometry) + + verify_value = verify_geometry + + elif isinstance(dataType, GeographyType): + + def verify_geography(obj: Any) -> None: + assert_acceptable_types(obj) + verify_acceptable_types(obj) + assert isinstance(obj, Geography) + + verify_value = verify_geography + else: def verify_default(obj: Any) -> None: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 212cc5db124ce..33622ca7349a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -29,9 +29,9 @@ import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData, STUtils} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{UTF8String, VariantVal} +import org.apache.spark.unsafe.types.{GeographyVal, GeometryVal, UTF8String, VariantVal} object EvaluatePython { @@ -43,7 +43,7 @@ object EvaluatePython { def needConversionInPython(dt: DataType): Boolean = dt match { case DateType | TimestampType | TimestampNTZType | VariantType | _: DayTimeIntervalType - | _: TimeType => true + | _: TimeType | _: GeometryType | _: GeographyType => true case _: StructType => true case _: UserDefinedType[_] => true case ArrayType(elementType, _) => needConversionInPython(elementType) @@ -92,6 +92,10 @@ object EvaluatePython { case (s: UTF8String, _: StringType) => s.toString + case (g: GeometryVal, gt: GeometryType) => STUtils.deserializeGeom(g, gt) + + case (g: GeographyVal, gt: GeographyType) => STUtils.deserializeGeog(g, gt) + case (bytes: Array[Byte], BinaryType) => if (binaryAsBytes) { new BytesWrapper(bytes) @@ -228,6 +232,23 @@ object EvaluatePython { ) } + case g: GeographyType => (obj: Any) => nullSafeConvert(obj) { + case s: java.util.HashMap[_, _] => + val geographySrid = s.get("srid").asInstanceOf[Int] + g.assertSridAllowedForType(geographySrid) + STUtils.stGeogFromWKB( + s.get("wkb").asInstanceOf[Array[Byte]]) + } + + case g: GeometryType => (obj: Any) => nullSafeConvert(obj) { + case s: java.util.HashMap[_, _] => + val geometrySrid = s.get("srid").asInstanceOf[Int] + g.assertSridAllowedForType(geometrySrid) + STUtils.stGeomFromWKB( + s.get("wkb").asInstanceOf[Array[Byte]], + geometrySrid) + } + case other => (obj: Any) => nullSafeConvert(obj)(PartialFunction.empty) } From 705a3a2f9f81824fb983111d90d59f3c48e427a0 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sat, 8 Nov 2025 07:45:14 -0800 Subject: [PATCH 079/400] [SPARK-54153][PYTHON] Support profiling iterator based Python UDFs ### What changes were proposed in this pull request? Updates the v2 Spark-session based Python UDF profiler to support profiling iterator based UDFs. ```python from collections.abc import Iterator from pstats import SortKey import pyarrow as pa df = spark.range(100000) def map_func(iter: Iterator[pa.RecordBatch]) -> Iterator[pa.RecordBatch]: for batch in iter: yield pa.RecordBatch.from_arrays([pa.compute.add(batch.column("id"), 10)], ["id"]) spark.conf.set('spark.sql.pyspark.udf.profiler', 'perf') df.mapInArrow(map_func, df.schema).collect() spark.conf.set('spark.sql.pyspark.udf.profiler', 'memory') df.mapInArrow(map_func, df.schema).collect() for stats in spark.profile.profiler_collector._perf_profile_results.values(): stats.sort_stats(SortKey.CUMULATIVE).print_stats(20) spark.profile.show(type="memory") ``` ``` 1395288 function calls (1359888 primitive calls) in 2.850 seconds Ordered by: cumulative time List reduced from 1546 to 20 due to restriction <20> ncalls tottime percall cumtime percall filename:lineno(function) 416 0.008 0.000 5.901 0.014 __init__.py:1() 424/24 0.000 0.000 2.850 0.119 {built-in method builtins.next} 24 0.001 0.000 2.850 0.119 test.py:11(map_func) 16 0.002 0.000 2.646 0.165 compute.py:244(wrapper) 2752/24 0.016 0.000 2.642 0.110 :1349(_find_and_load) 2752/24 0.013 0.000 2.641 0.110 :1304(_find_and_load_unlocked) 64 0.002 0.000 2.618 0.041 api.py:1() 2704/24 0.009 0.000 2.612 0.109 :911(_load_unlocked) 2264/24 0.005 0.000 2.611 0.109 :993(exec_module) 6336/48 0.004 0.000 2.591 0.054 :480(_call_with_frames_removed) 2400/24 0.023 0.000 2.591 0.108 {built-in method builtins.exec} 24 0.002 0.000 1.927 0.080 generic.py:1() 520/320 0.002 0.000 1.429 0.004 {built-in method builtins.__import__} 4312/2896 0.006 0.000 1.190 0.000 :1390(_handle_fromlist) 8 0.001 0.000 1.014 0.127 frame.py:1() 4392/4312 0.069 0.000 0.953 0.000 {built-in method builtins.__build_class__} 2264 0.030 0.000 0.562 0.000 :1066(get_code) 16 0.001 0.000 0.551 0.034 indexing.py:1() 24 0.001 0.000 0.451 0.019 datetimes.py:1() 16 0.001 0.000 0.401 0.025 datetimelike.py:1() ============================================================ Profile of UDF ============================================================ Filename: /data/projects/spark/python/test.py Line # Mem usage Increment Occurrences Line Contents ============================================================= 11 1212.1 MiB 1212.1 MiB 8 def map_func(iter: Iterator[pa.RecordBatch]) -> Iterator[pa.RecordBatch]: 12 1212.1 MiB -0.2 MiB 24 for batch in iter: 13 1212.1 MiB -0.2 MiB 32 yield pa.RecordBatch.from_arrays([pa.compute.add(batch.column("id"), 10)], ["id"]) ``` ### Why are the changes needed? To add valuable profiling support to all types of UDFs. ### Does this PR introduce _any_ user-facing change? Yes, iterator based Python UDFs can now be profiled with the SQL config based profiler. ### How was this patch tested? Updated UTs that were specifically testing that this wasn't supported to show they are now supported. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52853 from Kimahriman/udf-iter-profiler. Authored-by: Adam Binford Signed-off-by: Dongjoon Hyun (cherry picked from commit 6a369f953b0d134ed5acabe731005b234484b7ca) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_udf_profiler.py | 88 ++++++++++----- python/pyspark/tests/test_memory_profiler.py | 16 ++- python/pyspark/worker.py | 106 +++++++++++++----- .../python/UserDefinedPythonDataSource.scala | 3 +- .../python/MapInBatchEvaluatorFactory.scala | 5 +- .../sql/execution/python/MapInBatchExec.scala | 3 +- 6 files changed, 157 insertions(+), 64 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py index 4e8f722c22cbd..37f4a70fabd27 100644 --- a/python/pyspark/sql/tests/test_udf_profiler.py +++ b/python/pyspark/sql/tests/test_udf_profiler.py @@ -28,6 +28,7 @@ from pyspark import SparkConf from pyspark.errors import PySparkValueError from pyspark.sql import SparkSession +from pyspark.sql.datasource import DataSource, DataSourceReader from pyspark.sql.functions import col, arrow_udf, pandas_udf, udf from pyspark.sql.window import Window from pyspark.profiler import UDFBasicProfiler @@ -325,59 +326,47 @@ def add2(x): not have_pandas or not have_pyarrow, cast(str, pandas_requirement_message or pyarrow_requirement_message), ) - def test_perf_profiler_pandas_udf_iterator_not_supported(self): + def test_perf_profiler_pandas_udf_iterator(self): import pandas as pd @pandas_udf("long") - def add1(x): - return x + 1 - - @pandas_udf("long") - def add2(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: + def add(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: for s in iter: - yield s + 2 + yield s + 1 with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): - df = self.spark.range(10, numPartitions=2).select( - add1("id"), add2("id"), add1("id"), add2(col("id") + 1) - ) + df = self.spark.range(10, numPartitions=2).select(add("id")) df.collect() self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) for id in self.profile_results: - self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2) + self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=4) @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) - def test_perf_profiler_arrow_udf_iterator_not_supported(self): + def test_perf_profiler_arrow_udf_iterator(self): import pyarrow as pa @arrow_udf("long") - def add1(x): - return pa.compute.add(x, 1) - - @arrow_udf("long") - def add2(iter: Iterator[pa.Array]) -> Iterator[pa.Array]: + def add(iter: Iterator[pa.Array]) -> Iterator[pa.Array]: for s in iter: - yield pa.compute.add(s, 2) + yield pa.compute.add(s, 1) with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): - df = self.spark.range(10, numPartitions=2).select( - add1("id"), add2("id"), add1("id"), add2(col("id") + 1) - ) + df = self.spark.range(10, numPartitions=2).select(add("id")) df.collect() self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) for id in self.profile_results: - self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2) + self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=4) @unittest.skipIf( not have_pandas or not have_pyarrow, cast(str, pandas_requirement_message or pyarrow_requirement_message), ) - def test_perf_profiler_map_in_pandas_not_supported(self): - df = self.spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")) + def test_perf_profiler_map_in_pandas(self): + df = self.spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")).repartition(1) def filter_func(iterator): for pdf in iterator: @@ -386,7 +375,28 @@ def filter_func(iterator): with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): df.mapInPandas(filter_func, df.schema).show() - self.assertEqual(0, len(self.profile_results), str(self.profile_results.keys())) + self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) + + for id in self.profile_results: + self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2) + + @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) + def test_perf_profiler_map_in_arrow(self): + import pyarrow as pa + + df = self.spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")).repartition(1) + + def map_func(iterator: Iterator[pa.RecordBatch]) -> Iterator[pa.RecordBatch]: + for batch in iterator: + yield pa.RecordBatch.from_arrays( + [batch.column("id"), pa.compute.add(batch.column("age"), 1)], ["id", "age"] + ) + + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): + df.mapInArrow(map_func, df.schema).show() + + for id in self.profile_results: + self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2) @unittest.skipIf( not have_pandas or not have_pyarrow, @@ -575,6 +585,34 @@ def summarize(left, right): for id in self.profile_results: self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2) + def test_perf_profiler_data_source(self): + class TestDataSourceReader(DataSourceReader): + def __init__(self, schema): + self.schema = schema + + def partitions(self): + raise NotImplementedError + + def read(self, partition): + yield from ((1,), (2,), (3,)) + + class TestDataSource(DataSource): + def schema(self): + return "id long" + + def reader(self, schema) -> "DataSourceReader": + return TestDataSourceReader(schema) + + self.spark.dataSource.register(TestDataSource) + + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): + self.spark.read.format("TestDataSource").load().collect() + + self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) + + for id in self.profile_results: + self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=4) + def test_perf_profiler_render(self): with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): _do_computation(self.spark) diff --git a/python/pyspark/tests/test_memory_profiler.py b/python/pyspark/tests/test_memory_profiler.py index df9d63c5260f9..1909358aa2bc2 100644 --- a/python/pyspark/tests/test_memory_profiler.py +++ b/python/pyspark/tests/test_memory_profiler.py @@ -341,12 +341,13 @@ def add2(x): not have_pandas or not have_pyarrow, cast(str, pandas_requirement_message or pyarrow_requirement_message), ) - def test_memory_profiler_pandas_udf_iterator_not_supported(self): + def test_memory_profiler_pandas_udf_iterator(self): import pandas as pd @pandas_udf("long") - def add1(x): - return x + 1 + def add1(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: + for s in iter: + yield s + 1 @pandas_udf("long") def add2(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: @@ -359,7 +360,7 @@ def add2(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: ) df.collect() - self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) + self.assertEqual(3, len(self.profile_results), str(self.profile_results.keys())) for id in self.profile_results: self.assert_udf_memory_profile_present(udf_id=id) @@ -368,7 +369,7 @@ def add2(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: not have_pandas or not have_pyarrow, cast(str, pandas_requirement_message or pyarrow_requirement_message), ) - def test_memory_profiler_map_in_pandas_not_supported(self): + def test_memory_profiler_map_in_pandas(self): df = self.spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")) def filter_func(iterator): @@ -378,7 +379,10 @@ def filter_func(iterator): with self.sql_conf({"spark.sql.pyspark.udf.profiler": "memory"}): df.mapInPandas(filter_func, df.schema).show() - self.assertEqual(0, len(self.profile_results), str(self.profile_results.keys())) + self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) + + for id in self.profile_results: + self.assert_udf_memory_profile_present(udf_id=id) @unittest.skipIf( not have_pandas or not have_pyarrow, diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 09c6a40a33db9..6e34b041665ac 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1158,17 +1158,19 @@ def func(*args): return f, args_offsets -def _supports_profiler(eval_type: int) -> bool: - return eval_type not in ( +def _is_iter_based(eval_type: int) -> bool: + return eval_type in ( PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, PythonEvalType.SQL_MAP_ARROW_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE, + PythonEvalType.SQL_GROUPED_MAP_ARROW_ITER_UDF, + PythonEvalType.SQL_GROUPED_MAP_PANDAS_ITER_UDF, ) -def wrap_perf_profiler(f, result_id): +def wrap_perf_profiler(f, eval_type, result_id): import cProfile import pstats @@ -1178,38 +1180,89 @@ def wrap_perf_profiler(f, result_id): SpecialAccumulatorIds.SQL_UDF_PROFIER, None, ProfileResultsParam ) - def profiling_func(*args, **kwargs): - with cProfile.Profile() as pr: - ret = f(*args, **kwargs) - st = pstats.Stats(pr) - st.stream = None # make it picklable - st.strip_dirs() + if _is_iter_based(eval_type): + + def profiling_func(*args, **kwargs): + iterator = iter(f(*args, **kwargs)) + pr = cProfile.Profile() + while True: + try: + with pr: + item = next(iterator) + yield item + except StopIteration: + break + + st = pstats.Stats(pr) + st.stream = None # make it picklable + st.strip_dirs() + + accumulator.add({result_id: (st, None)}) - accumulator.add({result_id: (st, None)}) + else: + + def profiling_func(*args, **kwargs): + with cProfile.Profile() as pr: + ret = f(*args, **kwargs) + st = pstats.Stats(pr) + st.stream = None # make it picklable + st.strip_dirs() - return ret + accumulator.add({result_id: (st, None)}) + + return ret return profiling_func -def wrap_memory_profiler(f, result_id): +def wrap_memory_profiler(f, eval_type, result_id): from pyspark.sql.profiler import ProfileResultsParam from pyspark.profiler import UDFLineProfilerV2 + if not has_memory_profiler: + return f + accumulator = _deserialize_accumulator( SpecialAccumulatorIds.SQL_UDF_PROFIER, None, ProfileResultsParam ) - def profiling_func(*args, **kwargs): - profiler = UDFLineProfilerV2() + if _is_iter_based(eval_type): - wrapped = profiler(f) - ret = wrapped(*args, **kwargs) - codemap_dict = { - filename: list(line_iterator) for filename, line_iterator in profiler.code_map.items() - } - accumulator.add({result_id: (None, codemap_dict)}) - return ret + def profiling_func(*args, **kwargs): + profiler = UDFLineProfilerV2() + profiler.add_function(f) + + iterator = iter(f(*args, **kwargs)) + + while True: + try: + with profiler: + item = next(iterator) + yield item + except StopIteration: + break + + codemap_dict = { + filename: list(line_iterator) + for filename, line_iterator in profiler.code_map.items() + } + accumulator.add({result_id: (None, codemap_dict)}) + + else: + + def profiling_func(*args, **kwargs): + profiler = UDFLineProfilerV2() + profiler.add_function(f) + + with profiler: + ret = f(*args, **kwargs) + + codemap_dict = { + filename: list(line_iterator) + for filename, line_iterator in profiler.code_map.items() + } + accumulator.add({result_id: (None, codemap_dict)}) + return ret return profiling_func @@ -1254,17 +1307,12 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index, profil if profiler == "perf": result_id = read_long(infile) - if _supports_profiler(eval_type): - profiling_func = wrap_perf_profiler(chained_func, result_id) - else: - profiling_func = chained_func + profiling_func = wrap_perf_profiler(chained_func, eval_type, result_id) elif profiler == "memory": result_id = read_long(infile) - if _supports_profiler(eval_type) and has_memory_profiler: - profiling_func = wrap_memory_profiler(chained_func, result_id) - else: - profiling_func = chained_func + + profiling_func = wrap_memory_profiler(chained_func, eval_type, result_id) else: profiling_func = chained_func diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala index c147030037cd8..47e64a5b4041a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala @@ -172,7 +172,8 @@ case class UserDefinedPythonDataSource(dataSourceCls: PythonFunction) { pythonRunnerConf, metrics, jobArtifactUUID, - sessionUUID) + sessionUUID, + conf.pythonUDFProfiler) } def createPythonMetrics(): Array[CustomMetric] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchEvaluatorFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchEvaluatorFactory.scala index 4e78b3035a7ec..51909df26a567 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchEvaluatorFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchEvaluatorFactory.scala @@ -41,7 +41,8 @@ class MapInBatchEvaluatorFactory( pythonRunnerConf: Map[String, String], val pythonMetrics: Map[String, SQLMetric], jobArtifactUUID: Option[String], - sessionUUID: Option[String]) + sessionUUID: Option[String], + profiler: Option[String]) extends PartitionEvaluatorFactory[InternalRow, InternalRow] { override def createEvaluator(): PartitionEvaluator[InternalRow, InternalRow] = @@ -74,7 +75,7 @@ class MapInBatchEvaluatorFactory( pythonMetrics, jobArtifactUUID, sessionUUID, - None) with BatchedPythonArrowInput + profiler) with BatchedPythonArrowInput val columnarBatchIter = pyRunner.compute(batchIter, context.partitionId(), context) val unsafeProj = UnsafeProjection.create(output, output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchExec.scala index 1d03c0cf76037..c4f090674e7c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchExec.scala @@ -70,7 +70,8 @@ trait MapInBatchExec extends UnaryExecNode with PythonSQLMetrics { pythonRunnerConf, pythonMetrics, jobArtifactUUID, - sessionUUID) + sessionUUID, + conf.pythonUDFProfiler) val rdd = if (isBarrier) { val rddBarrier = child.execute().barrier() From 7499de3e2b4a7825f3f2ee883f44286a84a1ef35 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 8 Nov 2025 07:54:28 -0800 Subject: [PATCH 080/400] [SPARK-54239][BUILD] Bump Guava 33.4.8 ### What changes were proposed in this pull request? This PR bumps Guava from 33.4.0 to 33.4.8, the latest patch version of 33.4 serial. The release notes can be found at - https://github.com/google/guava/releases/tag/v33.4.8 - https://github.com/google/guava/releases/tag/v33.4.1 (this contains more info) As mentioned in the release notes, it migrates from jsr305 to jspecify, thus causes some transitive deps changes https://mvnrepository.com/artifact/com.google.guava/guava/33.4.0-jre https://mvnrepository.com/artifact/com.google.guava/guava/33.4.8-jre ```patch - com.google.code.findbugs:jsr305:3.0.2 - org.checkerframework:checker-qual:3.34.0 + org.jspecify:jspecify:1.0.0 ``` Note, the transitive deps change does not affect the Maven package because we use the white list to declare the included deps for shaded jars. ### Why are the changes needed? The current Guava 33.4.0 used by Spark was released in Dec 17, 2024. This upgrades it to the latest patched version (instead of 33.5.0) to make it align with gRPC 1.76.0 (this is not enforced, but nice to have) https://mvnrepository.com/artifact/io.grpc/grpc-api/1.76.0 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GHA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52939 from pan3793/guava-33.4.8. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 9289c4349ed3cb0f52b88c1777a531d0584e6a0c) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 4 ++-- pom.xml | 4 ++-- project/SparkBuild.scala | 16 ++++++++-------- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 3bf4d78f5023b..5d619db2a75e9 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -62,12 +62,12 @@ derbyshared/10.16.1.1//derbyshared-10.16.1.1.jar derbytools/10.16.1.1//derbytools-10.16.1.1.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar esdk-obs-java/3.20.4.2//esdk-obs-java-3.20.4.2.jar -failureaccess/1.0.2//failureaccess-1.0.2.jar +failureaccess/1.0.3//failureaccess-1.0.3.jar flatbuffers-java/25.2.10//flatbuffers-java-25.2.10.jar gcs-connector/hadoop3-2.2.28/shaded/gcs-connector-hadoop3-2.2.28-shaded.jar gmetric4j/1.0.10//gmetric4j-1.0.10.jar gson/2.11.0//gson-2.11.0.jar -guava/33.4.0-jre//guava-33.4.0-jre.jar +guava/33.4.8-jre//guava-33.4.8-jre.jar hadoop-aliyun/3.4.2//hadoop-aliyun-3.4.2.jar hadoop-annotations/3.4.2//hadoop-annotations-3.4.2.jar hadoop-aws/3.4.2//hadoop-aws-3.4.2.jar diff --git a/pom.xml b/pom.xml index d2469b11b3ab2..073ecf08b2e17 100644 --- a/pom.xml +++ b/pom.xml @@ -198,8 +198,8 @@ 2.12.1 4.1.17 - 33.4.0-jre - 1.0.2 + 33.4.8-jre + 1.0.3 2.11.0 3.1.9 3.0.18 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6b9bc8df5ce41..7a4c3f0828d03 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -881,17 +881,17 @@ object SparkConnectJdbc { // Exclude `scala-library` from assembly. (assembly / assemblyPackageScala / assembleArtifact) := false, - // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`, `jsr305-*.jar`, + // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`, `jspecify-*.jar`, // `error_prone_annotations-*.jar`, `listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar`, - // `j2objc-annotations-*.jar`, `checker-qual-*.jar` and `unused-1.0.0.jar` from assembly. + // `j2objc-annotations-*.jar` and `unused-1.0.0.jar` from assembly. (assembly / assemblyExcludedJars) := { val cp = (assembly / fullClasspath).value cp filter { v => val name = v.data.getName name.startsWith("pmml-model-") || name.startsWith("scala-collection-compat_") || - name.startsWith("jsr305-") || name.startsWith("error_prone_annotations") || + name.startsWith("jspecify-") || name.startsWith("error_prone_annotations") || name.startsWith("listenablefuture") || name.startsWith("j2objc-annotations") || - name.startsWith("checker-qual") || name == "unused-1.0.0.jar" + name == "unused-1.0.0.jar" } }, // Only include `spark-connect-client-jdbc-*.jar` @@ -971,17 +971,17 @@ object SparkConnectClient { // Exclude `scala-library` from assembly. (assembly / assemblyPackageScala / assembleArtifact) := false, - // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`, `jsr305-*.jar`, + // Exclude `pmml-model-*.jar`, `scala-collection-compat_*.jar`, `jspecify-*.jar`, // `error_prone_annotations-*.jar`, `listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar`, - // `j2objc-annotations-*.jar`, `checker-qual-*.jar` and `unused-1.0.0.jar` from assembly. + // `j2objc-annotations-*.jar` and `unused-1.0.0.jar` from assembly. (assembly / assemblyExcludedJars) := { val cp = (assembly / fullClasspath).value cp filter { v => val name = v.data.getName name.startsWith("pmml-model-") || name.startsWith("scala-collection-compat_") || - name.startsWith("jsr305-") || name.startsWith("error_prone_annotations") || + name.startsWith("jspecify-") || name.startsWith("error_prone_annotations") || name.startsWith("listenablefuture") || name.startsWith("j2objc-annotations") || - name.startsWith("checker-qual") || name == "unused-1.0.0.jar" + name == "unused-1.0.0.jar" } }, From 9d3e28f1945888540078d37fe0501805660a5633 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 8 Nov 2025 19:47:51 -0800 Subject: [PATCH 081/400] [SPARK-53944][K8S][CORE][FOLLOWUP] Override DRIVER_HOST_ADDRESS in SparkContext instead of SparkEnv when useDriverPodIP is true ### What changes were proposed in this pull request? This is an alternative to #52923. In addition, this ensures `spark.kubernetes.executor.useDriverPodIP` only takes effect when `spark.master` startsWith `k8s` ### Why are the changes needed? This is a simpler way than https://github.com/apache/spark/pull/52923, see more details at the discussion https://github.com/apache/spark/pull/52923#discussion_r2501500911 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the K8s IT case provided by #52923 ``` [info] KubernetesSuite: [info] - SPARK-53944: Run SparkPi without driver service (11 seconds, 914 milliseconds) [info] YuniKornSuite: [info] Run completed in 21 seconds, 737 milliseconds. [info] Total number of tests run: 1 [info] Suites: completed 2, aborted 0 [info] Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [success] Total time: 44 s, completed Nov 9, 2025, 1:19:07 AM ``` Also manually tested in the internal cluster. checklist: - `spark.kubernetes.executor.useDriverPodIP` only takes effect on K8s mode - when `s.k.e.useDriverPodIP` is true, check UI `Executors` and `Environment` tabs, driver address displays correctly in IP, not svc endpoint image image - run some queries, shuffle works as expected. (we use Apache Celeborn as Remote Shuffle Service) - all built-in functionalities that we used work with the driver svc disabled. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52954 from pan3793/SPARK-53944-refactor. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit cc57743a229a70e2060230e3c9da49cbe24ad257) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/SparkContext.scala | 9 ++++++++- core/src/main/scala/org/apache/spark/SparkEnv.scala | 11 +---------- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 898bbad26b7e3..2393851c66355 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -454,7 +454,14 @@ class SparkContext(config: SparkConf) extends Logging { // Set Spark driver host and port system properties. This explicitly sets the configuration // instead of relying on the default value of the config constant. - _conf.set(DRIVER_HOST_ADDRESS, _conf.get(DRIVER_HOST_ADDRESS)) + if (master.startsWith("k8s") && + _conf.getBoolean("spark.kubernetes.executor.useDriverPodIP", false)) { + logInfo("Use DRIVER_BIND_ADDRESS instead of DRIVER_HOST_ADDRESS as driver address " + + "because spark.kubernetes.executor.useDriverPodIP is true in K8s mode.") + _conf.set(DRIVER_HOST_ADDRESS, _conf.get(DRIVER_BIND_ADDRESS)) + } else { + _conf.set(DRIVER_HOST_ADDRESS, _conf.get(DRIVER_HOST_ADDRESS)) + } _conf.setIfMissing(DRIVER_PORT, 0) _conf.set(EXECUTOR_ID, SparkContext.DRIVER_IDENTIFIER) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 48d4faafb514e..796dbf4b6d5f8 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -263,9 +263,7 @@ object SparkEnv extends Logging { s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!") assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!") val bindAddress = conf.get(DRIVER_BIND_ADDRESS) - val useDriverPodIP = - conf.get("spark.kubernetes.executor.useDriverPodIP", "false").equalsIgnoreCase("true") - val advertiseAddress = if (useDriverPodIP) bindAddress else conf.get(DRIVER_HOST_ADDRESS) + val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS) val port = conf.get(DRIVER_PORT) val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(conf)) @@ -371,13 +369,6 @@ object SparkEnv extends Logging { logInfo(log"Registering ${MDC(LogKeys.ENDPOINT_NAME, name)}") rpcEnv.setupEndpoint(name, endpointCreator) } else { - val useDriverPodIP = - conf.get("spark.kubernetes.executor.useDriverPodIP", "false").equalsIgnoreCase("true") - if (useDriverPodIP) { - logInfo(log"Use DRIVER_BIND_ADDRESS instead of DRIVER_HOST_ADDRESS because " + - log"spark.kubernetes.executor.useDriverPodIP is true") - conf.set(config.DRIVER_HOST_ADDRESS.key, conf.get(config.DRIVER_BIND_ADDRESS.key)) - } RpcUtils.makeDriverRef(name, conf, rpcEnv) } } From c224a966726e9811bae299bfefeecbf5e7f871c6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 9 Nov 2025 18:28:56 -0800 Subject: [PATCH 082/400] [SPARK-54260][CORE][K8S] Unify K8s cluster checks to use the single regex via `SparkMasterRegex.isK8s` methods ### What changes were proposed in this pull request? This PR aims to unify all K8s cluster checks to use the single regex via `SparkMasterRegex.isK8s` methods. In addition, this PR adds a new Scalastyle rule to prevent a regression which adds this code pattern manually again. ### Why are the changes needed? Although Apache Spark has a clear definition of `K8s` cluster regex, there are many places to use improper string match patterns. We had better unity them to prevent human errors. **Official RegEx Definition** https://github.com/apache/spark/blob/cc57743a229a70e2060230e3c9da49cbe24ad257/core/src/main/scala/org/apache/spark/SparkContext.scala#L3461 **Manual String Match Logics** - Case 1: Only checking `k8s` prefix without checking `://`. ``` masterURL.startsWith("k8s") ``` ``` args.master.startsWith("k8s") ``` - Case 2: Checking `k8s://` via using a independent string which is error-prone. ``` master.startsWith("k8s://") ``` - Case 2: Checking with `Option` values which we can simplify more. ``` master.isDefined && master.get.startsWith("k8s://") ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs which ensures that all features are working correctly and pass the Scala linter. **BEFORE** ``` $ dev/scalastyle Scalastyle checks failed at following occurrences: [error] /Users/dongjoon/APACHE/spark-merge/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala:168:11: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesDiagnosticsSetter.scala:78:22: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala:38:65: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/util/Utils.scala:2881:25: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/SparkContext.scala:457:15: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala:173:56: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala:260:22: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala:1044:22: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala:52:53: Use SparkMasterRegex.isK8s instead. [error] /Users/dongjoon/APACHE/spark-merge/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala:182:78: Use SparkMasterRegex.isK8s instead. [error] Total time: 23 s, completed Nov 9, 2025, 12:03:39 PM ``` **AFTER** ``` $ dev/scalastyle Scalastyle checks passed. ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52957 from dongjoon-hyun/SPARK-54260. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 5a454f86cd5fd95c0ce30454a170ac5001813385) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/SparkContext.scala | 11 ++++++++++- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 6 +++--- .../org/apache/spark/resource/ResourceProfile.scala | 5 +++-- .../spark/resource/ResourceProfileManager.scala | 4 ++-- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- .../deploy/k8s/SparkKubernetesDiagnosticsSetter.scala | 4 ++-- .../apache/spark/deploy/k8s/submit/K8sSubmitOps.scala | 4 ++-- .../cluster/k8s/KubernetesClusterManager.scala | 2 +- scalastyle-config.xml | 5 +++++ 9 files changed, 29 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2393851c66355..6f8be49e3959b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -454,7 +454,7 @@ class SparkContext(config: SparkConf) extends Logging { // Set Spark driver host and port system properties. This explicitly sets the configuration // instead of relying on the default value of the config constant. - if (master.startsWith("k8s") && + if (SparkMasterRegex.isK8s(master) && _conf.getBoolean("spark.kubernetes.executor.useDriverPodIP", false)) { logInfo("Use DRIVER_BIND_ADDRESS instead of DRIVER_HOST_ADDRESS as driver address " + "because spark.kubernetes.executor.useDriverPodIP is true in K8s mode.") @@ -3459,6 +3459,15 @@ private object SparkMasterRegex { val SPARK_REGEX = """spark://(.*)""".r // Regular expression for connecting to kubernetes clusters val KUBERNETES_REGEX = """k8s://(.*)""".r + + def isK8s(master: String) : Boolean = isK8s(Option(master)) + + def isK8s(master: Option[String]) : Boolean = { + master match { + case Some(KUBERNETES_REGEX(_)) => true + case _ => false + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index b5d026e39a906..c3215b16f25e7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -170,7 +170,7 @@ private[spark] class SparkSubmit extends Logging { // Here we are checking for client mode because when job is sumbitted in cluster // deploy mode with k8s resource manager, the spark submit in the driver container // is done in client mode. - val isKubernetesClusterModeDriver = args.master.startsWith("k8s") && + val isKubernetesClusterModeDriver = SparkMasterRegex.isK8s(args.master) && "client".equals(args.deployMode) && sparkConf.getBoolean("spark.kubernetes.submitInDriver", false) if (isKubernetesClusterModeDriver) { @@ -257,7 +257,7 @@ private[spark] class SparkSubmit extends Logging { v match { case "yarn" => YARN case m if m.startsWith("spark") => STANDALONE - case m if m.startsWith("k8s") => KUBERNETES + case m if SparkMasterRegex.isK8s(m) => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => error("Master must either be yarn or start with spark, k8s, or local") @@ -1041,7 +1041,7 @@ private[spark] class SparkSubmit extends Logging { } throw cause } finally { - if (args.master.startsWith("k8s") && !isShell(args.primaryResource) && + if (SparkMasterRegex.isK8s(args.master) && !isShell(args.primaryResource) && !isSqlShell(args.mainClass) && !isThriftServer(args.mainClass) && !isConnectServer(args.mainClass)) { try { diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 1b4b4f61016a4..971b14265979e 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -25,6 +25,7 @@ import scala.collection.mutable import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException} +import org.apache.spark.SparkMasterRegex._ import org.apache.spark.annotation.{Evolving, Since} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ @@ -178,8 +179,8 @@ class ResourceProfile( // only applies to yarn/k8s private def shouldCheckExecutorCores(sparkConf: SparkConf): Boolean = { val master = sparkConf.getOption("spark.master") - sparkConf.contains(EXECUTOR_CORES) || - (master.isDefined && (master.get.equalsIgnoreCase("yarn") || master.get.startsWith("k8s"))) + sparkConf.contains(EXECUTOR_CORES) || isK8s(master) || + (master.isDefined && master.get.equalsIgnoreCase("yarn")) } /** diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index 10121f6ef2667..a3d76a92ddd8b 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -21,7 +21,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable.HashMap -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException, SparkMasterRegex} import org.apache.spark.annotation.Evolving import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys @@ -49,7 +49,7 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, private val dynamicEnabled = Utils.isDynamicAllocationEnabled(sparkConf) private val master = sparkConf.getOption("spark.master") private val isYarn = master.isDefined && master.get.equals("yarn") - private val isK8s = master.isDefined && master.get.startsWith("k8s://") + private val isK8s = SparkMasterRegex.isK8s(master) private val isStandaloneOrLocalCluster = master.isDefined && ( master.get.startsWith("spark://") || master.get.startsWith("local-cluster") ) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 81e86c82211c4..fc735ae3b99ea 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2878,7 +2878,7 @@ private[spark] object Utils * in canCreate to determine if the KubernetesClusterManager should be used. */ def checkAndGetK8sMasterUrl(rawMasterURL: String): String = { - require(rawMasterURL.startsWith("k8s://"), + require(SparkMasterRegex.isK8s(rawMasterURL), "Kubernetes master URL must start with k8s://.") val masterWithoutK8sPrefix = rawMasterURL.substring("k8s://".length) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesDiagnosticsSetter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesDiagnosticsSetter.scala index 9a1e79594e483..24f42919ce05a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesDiagnosticsSetter.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesDiagnosticsSetter.scala @@ -20,7 +20,7 @@ import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.hadoop.util.StringUtils -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkMasterRegex} import org.apache.spark.deploy.SparkDiagnosticsSetter import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants.EXIT_EXCEPTION_ANNOTATION @@ -75,6 +75,6 @@ private[spark] class SparkKubernetesDiagnosticsSetter(clientProvider: Kubernetes } override def supports(clusterManagerUrl: String): Boolean = { - clusterManagerUrl.startsWith("k8s://") + SparkMasterRegex.isK8s(clusterManagerUrl) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala index 17704b908558e..bd8e0f97132dd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala @@ -23,7 +23,7 @@ import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkMasterRegex} import org.apache.spark.deploy.SparkSubmitOperation import org.apache.spark.deploy.k8s.{KubernetesUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config.{KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, KUBERNETES_SUBMIT_GRACE_PERIOD} @@ -165,7 +165,7 @@ private[spark] class K8SSparkSubmitOperation extends SparkSubmitOperation } override def supports(master: String): Boolean = { - master.startsWith("k8s://") + SparkMasterRegex.isK8s(master) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 49eac64745b7c..3fb1ed0c9c0fc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { import SparkMasterRegex._ - override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") + override def canCreate(masterURL: String): Boolean = SparkMasterRegex.isK8s(masterURL) private def isLocal(conf: SparkConf): Boolean = conf.get(KUBERNETES_DRIVER_MASTER_URL).startsWith("local") diff --git a/scalastyle-config.xml b/scalastyle-config.xml index cea4cd3ac4fc1..9bcdecbbf4fdc 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -846,4 +846,9 @@ This file is divided into 3 sections: \bInts\.checkedCast\b Use JavaUtils.checkedCast instead. + + + \bstartsWith\("k8s\b + Use SparkMasterRegex.isK8s instead. + From 4e9ebd414966b1033f97fd0faa7d07739043c61c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 9 Nov 2025 18:33:23 -0800 Subject: [PATCH 083/400] [SPARK-49862][INFRA][FOLLOWUP] Update `dev/infra/Dockerfile` too ### What changes were proposed in this pull request? This is a follow-up of the following to update `dev/infra/Dockerfile` together. - https://github.com/apache/spark/pull/50539 ### Why are the changes needed? The original goal of SPARK-49862 was updating both files. ### Does this PR introduce _any_ user-facing change? No behavior change because this is an infra change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52952 from dongjoon-hyun/SPARK-49862. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 5c11509d4ef3a25c390fb0fa433473a67a71799f) Signed-off-by: Dongjoon Hyun --- dev/infra/Dockerfile | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index fdfd89bdf06d3..655e93d9eecc0 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -147,9 +147,10 @@ RUN apt-get update && apt-get install -y \ python3.13 \ && rm -rf /var/lib/apt/lists/* RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 -# TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 RUN python3.13 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this -RUN python3.13 -m pip install 'numpy>=2.1' 'pyarrow>=18.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy coverage matplotlib openpyxl 'grpcio==1.76.0' 'grpcio-status==1.76.0' lxml jinja2 && \ +RUN python3.13 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ + python3.13 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ + python3.13 -m pip install torcheval && \ python3.13 -m pip cache purge # Remove unused installation packages to free up disk space From 04838949975cd1357cf5ac41ebcd33fde15e7a91 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 9 Nov 2025 18:34:12 -0800 Subject: [PATCH 084/400] [SPARK-54263][INFRA] Fix `run-in-container` script to use Python 3.11 instead of 3.9 ### What changes were proposed in this pull request? This PR aims to fix `run-in-container` script to use Python 3.11 instead of 3.9. ### Why are the changes needed? Since Apache Spark 4.0.0, we have an optional way to build docs in Docker image. - #48860 https://github.com/apache/spark/blob/fa33ea000a0bda9e5a3fa1af98e8e85b8cc5e4d4/docs/README.md?plain=1#L129-L136 When we started to use Python 3.11 instead of 3.9 for Apache Spark 4.1.0, we missed to update this `run-in-container`. - #51150 - #51198 ### Does this PR introduce _any_ user-facing change? No behavior change because this is an optional way to generate Spark docs. ### How was this patch tested? Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52959 from dongjoon-hyun/SPARK-54263. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 6466ed579d5aeca31c94d0a0dbbad00865caa2c3) Signed-off-by: Dongjoon Hyun --- dev/spark-test-image-util/docs/run-in-container | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/spark-test-image-util/docs/run-in-container b/dev/spark-test-image-util/docs/run-in-container index 1d43c602f7c72..3bfb3c5f651dd 100644 --- a/dev/spark-test-image-util/docs/run-in-container +++ b/dev/spark-test-image-util/docs/run-in-container @@ -28,8 +28,8 @@ cd /__w/spark/spark/docs bundle install # 3.Build docs, includes: `error docs`, `scala doc`, `python doc`, `sql doc`, excludes: `r doc`. -# We need this link to make sure `python3` points to `python3.9` which contains the prerequisite packages. -ln -s "$(which python3.9)" "/usr/local/bin/python3" +# We need this link to make sure `python3` points to `python3.11` which contains the prerequisite packages. +ln -s "$(which python3.11)" "/usr/local/bin/python3" # Build docs first with SKIP_API to ensure they are buildable without requiring any # language docs to be built beforehand. From ebe86fa810c91fb634367b25f947467d59db54a4 Mon Sep 17 00:00:00 2001 From: cty123 Date: Mon, 10 Nov 2025 12:41:40 +0800 Subject: [PATCH 085/400] [SPARK-54205][CONNECT] Supports Decimal type data in SparkConnectResultSet ### What changes were proposed in this pull request? Spark connect has supported JDBC protocol with a few commonly used SQL data types. But currently it's missing the support for Decimal data which is also very commonly used to store money objects. I would like to have it support Decimal data type. ### Why are the changes needed? Right now, a user is able to read Decimal data from SQL by converting the data to string, and then parse the string into Java BigDecimal object. But since JDBC driver is already able to fetch the data as Java BigDecimal type, we can save the effort converting it back and forth. Instead, we just pass through the data we obtain from the raw JDBC result set. ### Does this PR introduce _any_ user-facing change? It's part of a new feature under Spark connect JDBC support. ### How was this patch tested? I have created a test new unit test named **'get decimal type'** and it covers my changes. Also the test case aligns with the tests for fetching other data types. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52947 from cty123/cty123/support-spark-connect-decimaltype. Lead-authored-by: cty123 Co-authored-by: cty Signed-off-by: yangjie01 (cherry picked from commit 73b50e9163a1cd6ca633ecc2e443ce61464115d5) Signed-off-by: yangjie01 --- .../client/jdbc/SparkConnectResultSet.scala | 12 +++++-- .../client/jdbc/util/JdbcTypeUtils.scala | 14 +++++++- .../jdbc/SparkConnectJdbcDataTypeSuite.scala | 33 +++++++++++++++++++ 3 files changed, 55 insertions(+), 4 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index 0745ddc099111..23c2315400fff 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -257,11 +257,17 @@ class SparkConnectResultSet( override def getCharacterStream(columnLabel: String): Reader = throw new SQLFeatureNotSupportedException - override def getBigDecimal(columnIndex: Int): java.math.BigDecimal = - throw new SQLFeatureNotSupportedException + override def getBigDecimal(columnIndex: Int): java.math.BigDecimal = { + if (currentRow.isNullAt(columnIndex - 1)) { + _wasNull = true + return null + } + _wasNull = false + currentRow.getDecimal(columnIndex - 1) + } override def getBigDecimal(columnLabel: String): java.math.BigDecimal = - throw new SQLFeatureNotSupportedException + getBigDecimal(findColumn(columnLabel)) override def isBeforeFirst: Boolean = { checkOpen() diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala index 55e3d29c99a5e..c2b27128caa72 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connect.client.jdbc.util import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Long => JLong, Short => JShort} +import java.math.{BigDecimal => JBigDecimal} import java.sql.{Array => _, _} import org.apache.spark.sql.types._ @@ -34,6 +35,7 @@ private[jdbc] object JdbcTypeUtils { case FloatType => Types.FLOAT case DoubleType => Types.DOUBLE case StringType => Types.VARCHAR + case _: DecimalType => Types.DECIMAL case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -48,12 +50,14 @@ private[jdbc] object JdbcTypeUtils { case FloatType => classOf[JFloat].getName case DoubleType => classOf[JDouble].getName case StringType => classOf[String].getName + case _: DecimalType => classOf[JBigDecimal].getName case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } def isSigned(field: StructField): Boolean = field.dataType match { - case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType => true + case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | + _: DecimalType => true case NullType | BooleanType | StringType => false case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") @@ -69,6 +73,7 @@ private[jdbc] object JdbcTypeUtils { case FloatType => 7 case DoubleType => 15 case StringType => 255 + case DecimalType.Fixed(p, _) => p case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -77,6 +82,7 @@ private[jdbc] object JdbcTypeUtils { case FloatType => 7 case DoubleType => 15 case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | StringType => 0 + case DecimalType.Fixed(_, s) => s case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -90,6 +96,12 @@ private[jdbc] object JdbcTypeUtils { case DoubleType => 24 case StringType => getPrecision(field) + // precision + negative sign + leading zero + decimal point, like DECIMAL(5,5) = -0.12345 + case DecimalType.Fixed(p, s) if p == s => p + 3 + // precision + negative sign, like DECIMAL(5,0) = -12345 + case DecimalType.Fixed(p, s) if s == 0 => p + 1 + // precision + negative sign + decimal point, like DECIMAL(5,2) = -123.45 + case DecimalType.Fixed(p, _) => p + 2 case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index 619b279310eb3..089c1d7fdf0d4 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -215,4 +215,37 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess assert(metaData.getColumnDisplaySize(1) === 255) } } + + test("get decimal type") { + Seq( + ("123.45", 37, 2, 39), + ("-0.12345", 5, 5, 8), + ("-0.12345", 6, 5, 8), + ("-123.45", 5, 2, 7), + ("12345", 5, 0, 6), + ("-12345", 5, 0, 6) + ).foreach { + case (value, precision, scale, expectedColumnDisplaySize) => + val decimalType = s"DECIMAL($precision,$scale)" + withExecuteQuery(s"SELECT cast('$value' as $decimalType)") { rs => + assert(rs.next()) + assert(rs.getBigDecimal(1) === new java.math.BigDecimal(value)) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === s"CAST($value AS $decimalType)") + assert(metaData.getColumnLabel(1) === s"CAST($value AS $decimalType)") + assert(metaData.getColumnType(1) === Types.DECIMAL) + assert(metaData.getColumnTypeName(1) === decimalType) + assert(metaData.getColumnClassName(1) === "java.math.BigDecimal") + assert(metaData.isSigned(1) === true) + assert(metaData.getPrecision(1) === precision) + assert(metaData.getScale(1) === scale) + assert(metaData.getColumnDisplaySize(1) === expectedColumnDisplaySize) + assert(metaData.getColumnDisplaySize(1) >= value.size) + } + } + } } From e1e42386bdd18c980ccb6144bc548dcf81c2cd66 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 9 Nov 2025 21:29:00 -0800 Subject: [PATCH 086/400] [SPARK-54226][SQL][FOLLOWUP] Remove zstd compression level value check and update config names ### What changes were proposed in this pull request? This is a followup to https://github.com/apache/spark/pull/52925. ### Why are the changes needed? To address review comments. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Code v2.0.14 Closes #52962 from viirya/arrow_compress_udf_followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun (cherry picked from commit 020c1bd2b6356bd8a82e2eaf993a6d141e71ab87) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/arrow/test_arrow.py | 8 ++++---- .../sql/tests/pandas/test_pandas_grouped_map.py | 6 +++--- .../tests/pandas/test_pandas_udf_grouped_agg.py | 4 ++-- .../sql/tests/pandas/test_pandas_udf_scalar.py | 6 +++--- .../org/apache/spark/sql/internal/SQLConf.scala | 15 +++++++-------- 5 files changed, 19 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow.py b/python/pyspark/sql/tests/arrow/test_arrow.py index af08f8c8c1012..14f8fbe33c8e1 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow.py +++ b/python/pyspark/sql/tests/arrow/test_arrow.py @@ -1817,7 +1817,7 @@ def test_toPandas_with_compression_codec(self): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): pdf = df.toPandas() assert_frame_equal(expected, pdf) @@ -1846,7 +1846,7 @@ def test_toArrow_with_compression_codec(self): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): t_out = df.toArrow() self.assertTrue(t_out.equals(t_in)) @@ -1863,7 +1863,7 @@ def test_toPandas_with_compression_codec_large_dataset(self): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): pdf = df.toPandas() self.assertEqual(len(pdf), 10000) self.assertEqual(pdf.columns.tolist(), ["id", "str_col", "mod_col"]) @@ -1880,7 +1880,7 @@ def test_toArrow_with_compression_codec_large_dataset(self): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): t = df.toArrow() self.assertEqual(t.num_rows, 10000) self.assertEqual(t.column_names, ["id", "str_col", "mod_col"]) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index b60c5a187fbf7..a2f31bacf8127 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -1418,7 +1418,7 @@ def foo(pdf): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = df.groupby("id").apply(foo).sort("id").toPandas() assert_frame_equal(expected, result) @@ -1432,7 +1432,7 @@ def stats(key, pdf): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = ( df.groupby("id") .applyInPandas(stats, schema="id long, mean double") @@ -1457,7 +1457,7 @@ def sum_func(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = ( df.groupby("id") .applyInPandas(sum_func, schema="v double") diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py index 2958d0e67f1ea..6915e8aee948d 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py @@ -872,7 +872,7 @@ def sum_udf(v): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = df.groupby("id").agg(sum_udf(df.v)).sort("id").toPandas() assert_frame_equal(expected, result) @@ -891,7 +891,7 @@ def sum_udf(v): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = ( df.groupby("id").agg(mean_udf(df.v), sum_udf(df.v)).sort("id").toPandas() ) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py index 554c994afc1ed..563e0b789a962 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py @@ -1999,7 +1999,7 @@ def plus_one(v): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = df.select(plus_one("id").alias("result")).collect() self.assertEqual(expected, result) @@ -2017,7 +2017,7 @@ def create_array(v): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): # Test string UDF result = df.select(concat_string("id").alias("result")).collect() expected = [Row(result=f"value_{i}") for i in range(50)] @@ -2040,7 +2040,7 @@ def plus_two(iterator): for codec in ["none", "zstd", "lz4"]: with self.subTest(compressionCodec=codec): - with self.sql_conf({"spark.sql.execution.arrow.compressionCodec": codec}): + with self.sql_conf({"spark.sql.execution.arrow.compression.codec": codec}): result = df.select(plus_two("id").alias("result")).collect() self.assertEqual(expected, result) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8f27c9097d3d0..2316acaa173e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4002,7 +4002,7 @@ object SQLConf { .createWithDefaultString("64MB") val ARROW_EXECUTION_COMPRESSION_CODEC = - buildConf("spark.sql.execution.arrow.compressionCodec") + buildConf("spark.sql.execution.arrow.compression.codec") .doc("Compression codec used to compress Arrow IPC data when transferring data " + "between JVM and Python processes (e.g., toPandas, toArrow). This can significantly " + "reduce memory usage and network bandwidth when transferring large datasets. " + @@ -4016,16 +4016,15 @@ object SQLConf { .createWithDefault("none") val ARROW_EXECUTION_ZSTD_COMPRESSION_LEVEL = - buildConf("spark.sql.execution.arrow.zstd.compressionLevel") + buildConf("spark.sql.execution.arrow.compression.zstd.level") .doc("Compression level for Zstandard (zstd) codec when compressing Arrow IPC data. " + - "This config is only used when spark.sql.execution.arrow.compressionCodec is set to " + - "'zstd'. Valid values are integers from 1 (fastest, lowest compression) to 22 " + - "(slowest, highest compression). The default value 3 provides a good balance between " + - "compression speed and compression ratio.") + "This config is only used when spark.sql.execution.arrow.compression.codec is set to " + + "'zstd'. Negative values provide ultra-fast compression with lower " + + "compression ratios. Positive values provide normal to maximum compression, " + + "with higher values giving better compression but slower speed. The default value 3 " + + "provides a good balance between compression speed and compression ratio.") .version("4.1.0") .intConf - .checkValue(level => level >= 1 && level <= 22, - "Zstd compression level must be between 1 and 22") .createWithDefault(3) val ARROW_TRANSFORM_WITH_STATE_IN_PYSPARK_MAX_STATE_RECORDS_PER_BATCH = From 0fb91ff2d2118b37bdbe511fed3c5320489c2f90 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 9 Nov 2025 22:41:26 -0800 Subject: [PATCH 087/400] [SPARK-54269][PYTHON] Upgrade `cloudpickle` to 3.1.2 for Python 3.14 ### What changes were proposed in this pull request? This PR aims to upgrade `cloudpickle` to 3.1.2. ### Why are the changes needed? To support Python 3.14 properly. - https://github.com/cloudpipe/cloudpickle/releases/tag/v3.1.2 - https://github.com/cloudpipe/cloudpickle/blob/master/CHANGES.md#312 > Fix pickling of abstract base classes containing type annotations for Python 3.14. (https://github.com/cloudpipe/cloudpickle/pull/578) ### Does this PR introduce _any_ user-facing change? No, Python 3.14 support is not announced yet. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52964 from dongjoon-hyun/SPARK-54269. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 9754e1750614ab05bcf79055f3ac1d883314ebb9) Signed-off-by: Dongjoon Hyun --- python/pyspark/cloudpickle/__init__.py | 2 +- python/pyspark/cloudpickle/cloudpickle.py | 19 +++++++++++++------ 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py index bdb1738611b3b..052b6e975a772 100644 --- a/python/pyspark/cloudpickle/__init__.py +++ b/python/pyspark/cloudpickle/__init__.py @@ -3,7 +3,7 @@ __doc__ = cloudpickle.__doc__ -__version__ = "3.1.1" +__version__ = "3.1.2" __all__ = [ # noqa "__version__", diff --git a/python/pyspark/cloudpickle/cloudpickle.py b/python/pyspark/cloudpickle/cloudpickle.py index 4d532e5de9f2c..e600b35f28422 100644 --- a/python/pyspark/cloudpickle/cloudpickle.py +++ b/python/pyspark/cloudpickle/cloudpickle.py @@ -783,6 +783,12 @@ def _class_getstate(obj): clsdict.pop("__dict__", None) # unpicklable property object + if sys.version_info >= (3, 14): + # PEP-649/749: __annotate_func__ contains a closure that references the class + # dict. We need to exclude it from pickling. Python will recreate it when + # __annotations__ is accessed at unpickling time. + clsdict.pop("__annotate_func__", None) + return (clsdict, {}) @@ -1190,6 +1196,10 @@ def _class_setstate(obj, state): for subclass in registry: obj.register(subclass) + # PEP-649/749: During pickling, we excluded the __annotate_func__ attribute but it + # will be created by Python. Subsequently, annotations will be recreated when + # __annotations__ is accessed. + return obj @@ -1301,12 +1311,9 @@ def _function_getnewargs(self, func): def dump(self, obj): try: return super().dump(obj) - except RuntimeError as e: - if len(e.args) > 0 and "recursion" in e.args[0]: - msg = "Could not pickle object as excessively deep recursion required." - raise pickle.PicklingError(msg) from e - else: - raise + except RecursionError as e: + msg = "Could not pickle object as excessively deep recursion required." + raise pickle.PicklingError(msg) from e def __init__(self, file, protocol=None, buffer_callback=None): if protocol is None: From 8c087f1a1ecc1749cbab4300882f15ab0b5e7744 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 9 Nov 2025 23:11:35 -0800 Subject: [PATCH 088/400] [SPARK-54065][PYTHON][TESTS] Enable `test_in_memory_data_source` in Python 3.14 ### What changes were proposed in this pull request? This PR aims to enable `test_in_memory_data_source` in Python 3.14. ### Why are the changes needed? After upgrading to the latest `cloudpickle` 3.1.2, I verified that the test passed on Python 3.14.0. - #52964 ``` $ python/run-tests --parallelism 1 --testnames pyspark.sql.tests.test_python_datasource --python-executables python3 Running PySpark tests. Output is in /Users/dongjoon/APACHE/spark-merge/python/unit-tests.log Will test against the following Python executables: ['python3'] Will test the following Python tests: ['pyspark.sql.tests.test_python_datasource'] python3 python_implementation is CPython python3 version is: Python 3.14.0 Starting test(python3): pyspark.sql.tests.test_python_datasource (temp output: /Users/dongjoon/APACHE/spark-merge/python/target/e771105c-5304-4f4f-bea6-b8d896b1b250/python3__pyspark.sql.tests.test_python_datasource__2wyrv5n4.log) Finished test(python3): pyspark.sql.tests.test_python_datasource (29s) Tests passed in 29 seconds ``` ### Does this PR introduce _any_ user-facing change? No. This is a test change. ### How was this patch tested? Pass the CIs and manual tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52967 from dongjoon-hyun/SPARK-54065. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 79d0c427878f3fc9a95375679934ce6d59595014) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_python_datasource.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_python_datasource.py b/python/pyspark/sql/tests/test_python_datasource.py index cfedf1cf075ba..ee2d0ac60d90d 100644 --- a/python/pyspark/sql/tests/test_python_datasource.py +++ b/python/pyspark/sql/tests/test_python_datasource.py @@ -258,7 +258,6 @@ def test_read_with_invalid_return_row_type(self): with self.assertRaisesRegex(PythonException, "DATA_SOURCE_INVALID_RETURN_TYPE"): df.collect() - @unittest.skipIf(sys.version_info > (3, 13), "SPARK-54065") def test_in_memory_data_source(self): class InMemDataSourceReader(DataSourceReader): DEFAULT_NUM_PARTITIONS: int = 3 From 9814a089deaba758655c7f3570b71d643cf2f4eb Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 10 Nov 2025 17:50:11 +0800 Subject: [PATCH 089/400] [SPARK-54065][PYTHON][TESTS][FOLLOWUP] Remove unused import module from `test_python_datasource.py` ### What changes were proposed in this pull request? This PR aims to recover CI by removing `import sys` from `test_python_datasource.py` After #52967, CI consistently fails because `sys` is no longer used. ### Why are the changes needed? To recover CI. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52970 from sarutak/followup-SPARK-54065. Authored-by: Kousuke Saruta Signed-off-by: Ruifeng Zheng (cherry picked from commit 8cc76fe19606c9d0be80757bea31aab14024e5dc) Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/tests/test_python_datasource.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_python_datasource.py b/python/pyspark/sql/tests/test_python_datasource.py index ee2d0ac60d90d..532f68f2c8f26 100644 --- a/python/pyspark/sql/tests/test_python_datasource.py +++ b/python/pyspark/sql/tests/test_python_datasource.py @@ -16,7 +16,6 @@ # import os import platform -import sys import tempfile import unittest import logging From 2e747d69d4e11b43f53dc8f7328640d69fe1a7d3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 10 Nov 2025 08:55:11 -0800 Subject: [PATCH 090/400] [SPARK-54277][INFRA] Make `dev/run-tests` to ban `Python 3.9` and older versions ### What changes were proposed in this pull request? This PR aims to make `dev/run-tests` to ban Python 3.9 and older versions. ### Why are the changes needed? This will prevent users from running Python 3.9 and older versions for testing. ``` $ python3 --version Python 3.9.22 $ ./python/run-tests --python-executable python3 Python versions prior to 3.10 are not supported. ``` ### Does this PR introduce _any_ user-facing change? No behavior change because this is a test change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52974 from dongjoon-hyun/SPARK-54277. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit acfedacd564bdd9a7c0113d1521e1e9bc633420c) Signed-off-by: Dongjoon Hyun --- dev/run-tests | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 91a1532a338b9..6067caf210ebb 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -20,9 +20,9 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 8, 0))') +PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 10, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 3.8 are not supported." + echo "Python versions prior to 3.10 are not supported." exit -1 fi From 34102a5630d12fd5c61558fe7c3915c40d86540d Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 10 Nov 2025 09:58:51 -0800 Subject: [PATCH 091/400] [SPARK-54275][SQL] Clean up unused code from `pipeline` module ### What changes were proposed in this pull request? This pr aims to Clean up the unused code from the pipeline module before the release of Apache Spark 4.1.0. ### Why are the changes needed? Code clean up. ### Does this PR introduce _any_ user-facing change? No, this is a new module added in the 4.1 cycle. ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #52972 from LuciferYang/pipeline-clean. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun (cherry picked from commit c477ee6bd3c74e96f2ef85dcf0ab273099b99979) Signed-off-by: Dongjoon Hyun --- .../graph/DataflowGraphTransformer.scala | 8 --- .../sql/pipelines/graph/GraphErrors.scala | 19 ------- .../sql/pipelines/logging/EventHelpers.scala | 52 ------------------- .../sql/pipelines/utils/PipelineTest.scala | 5 -- 4 files changed, 84 deletions(-) delete mode 100644 sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/EventHelpers.scala diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala index 4121591c46b2d..c80978a5957dd 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala @@ -109,14 +109,6 @@ class DataflowGraphTransformer(graph: DataflowGraph) extends AutoCloseable { this } - private def defaultOnFailedDependentTables( - failedTableDependencies: Map[TableIdentifier, Seq[Table]]): Unit = { - require( - failedTableDependencies.isEmpty, - "Dependency failure happened and some tables were not resolved" - ) - } - /** * Example graph: [Flow1, Flow 2] -> ST -> Flow3 -> MV * Order of processing: Flow1, Flow2, ST, Flow3, MV. diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphErrors.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphErrors.scala index 53db669e687d2..c835665a0f380 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphErrors.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphErrors.scala @@ -38,25 +38,6 @@ object GraphErrors { ) } - /** - * Throws when the catalog or schema name in the "USE CATALOG | SCHEMA" command is invalid - * - * @param command string "USE CATALOG" or "USE SCHEMA" - * @param name the invalid catalog or schema name - * @param reason the reason why the name is invalid - */ - def invalidNameInUseCommandError( - command: String, - name: String, - reason: String - ): SparkException = { - new SparkException( - errorClass = "INVALID_NAME_IN_USE_COMMAND", - messageParameters = Map("command" -> command, "name" -> name, "reason" -> reason), - cause = null - ) - } - /** * Throws when a table path is unresolved, i.e. the table identifier * does not exist in the catalog. diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/EventHelpers.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/EventHelpers.scala deleted file mode 100644 index 37f8dc67c385c..0000000000000 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/EventHelpers.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.pipelines.logging - -import java.sql.Timestamp -import java.time.{Instant, ZoneId} -import java.time.format.DateTimeFormatter - -/** Contains helpers and implicits for working with [[PipelineEvent]]s. */ -object EventHelpers { - - /** A format string that defines how timestamps are serialized in a [[PipelineEvent]]. */ - private val timestampFormat: String = "yyyy-MM-dd'T'HH:mm:ss.SSSXX" - // Currently only the UTC timezone is supported. Eventually we want to allow the user to specify - // the timezone as a pipeline level setting using the SESSION_LOCAL_TIMEZONE key, and it should - // not be possible to change this setting during a pipeline run. - private val zoneId: ZoneId = ZoneId.of("UTC") - - private val formatter: DateTimeFormatter = DateTimeFormatter - .ofPattern(timestampFormat) - .withZone(zoneId) - - /** Converts a timestamp to a string in ISO 8601 format. */ - def formatTimestamp(ts: Timestamp): String = { - val instant = Instant.ofEpochMilli(ts.getTime) - formatter.format(instant) - } - - /** Converts an ISO 8601 formatted timestamp to a {@link java.sql.Timestamp}. */ - def parseTimestamp(timeString: String): Timestamp = { - if (timeString.isEmpty) { - new Timestamp(0L) - } else { - val instant = Instant.from(formatter.parse(timeString)) - new Timestamp(instant.toEpochMilli) - } - } -} diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala index f9d6aba9e22d1..fd97570bc528a 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala @@ -231,11 +231,6 @@ abstract class PipelineTest ignoreFieldCase: Boolean = false ) - /** Holds a parsed version along with the original json of a test. */ - private case class TestSequence(json: Seq[String], rows: Seq[Row]) { - require(json.size == rows.size) - } - /** * Helper method to verify unresolved column error message. We expect three elements to be present * in the message: error class, unresolved column name, list of suggested columns. From 9beab99e298e9ae9856e0789872bf7979a6b4fba Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 10 Nov 2025 10:02:39 -0800 Subject: [PATCH 092/400] [SPARK-54278][PYTHON][TESTS] Simplify `pyspark/util.py` doctests by removing Python 3.9+ condition ### What changes were proposed in this pull request? This PR aims to simplify `pyspark/util.py` doctests by removing Python 3.9+ condition. ### Why are the changes needed? Since Apache Spark 4.1.0 dropped `Python 3.9` support, we can assume that this doctests are Python 3.10+ always. - #51416 - #51259 - #52974 ### Does this PR introduce _any_ user-facing change? No, this is a test change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52975 from dongjoon-hyun/SPARK-54278. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit dbc643ee84f8cebeeb0b0b5bfe8df06ff136ac93) Signed-off-by: Dongjoon Hyun --- python/pyspark/util.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/util.py b/python/pyspark/util.py index f633ed699ee2d..501d0afbb163f 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -918,7 +918,7 @@ def default_api_mode() -> str: if __name__ == "__main__": - if "pypy" not in platform.python_implementation().lower() and sys.version_info[:2] >= (3, 9): + if "pypy" not in platform.python_implementation().lower(): import doctest import pyspark.util from pyspark.core.context import SparkContext From 4c498c3f0cd2e3df8d9d0fd0f54aac9628ccc419 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 10 Nov 2025 10:04:33 -0800 Subject: [PATCH 093/400] [SPARK-54112][CONNECT] Support getSchemas for SparkConnectDatabaseMetaData ### What changes were proposed in this pull request? Implement `getSchemas` methods defined in `java.sql.DatabaseMetaData` for `SparkConnectDatabaseMetaData`. ```java /** * Retrieves the schema names available in this database. The results * are ordered by {code TABLE_CATALOG} and * {code TABLE_SCHEM}. * *

The schema columns are: *

    *
  1. TABLE_SCHEM String {code =>} schema name *
  2. TABLE_CATALOG String {code =>} catalog name (may be {code null}) *
* * return a {code ResultSet} object in which each row is a * schema description * throws SQLException if a database access error occurs * */ ResultSet getSchemas() throws SQLException; /** * Retrieves the schema names available in this database. The results * are ordered by {code TABLE_CATALOG} and * {code TABLE_SCHEM}. * *

The schema columns are: *

    *
  1. TABLE_SCHEM String {code =>} schema name *
  2. TABLE_CATALOG String {code =>} catalog name (may be {code null}) *
* * * param catalog a catalog name; must match the catalog name as it is stored * in the database;"" retrieves those without a catalog; null means catalog * name should not be used to narrow down the search. * param schemaPattern a schema name; must match the schema name as it is * stored in the database; null means * schema name should not be used to narrow down the search. * return a {code ResultSet} object in which each row is a * schema description * throws SQLException if a database access error occurs * see #getSearchStringEscape * since 1.6 */ ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException; ``` ### Why are the changes needed? Enhance API coverage of the Connect JDBC driver, for example, `get[Catalogs|Schemas|Tables|...]` APIs are used by SQL GUI tools such as DBeaver for displaying the tree category. ### Does this PR introduce _any_ user-facing change? No, Connect JDBC driver is a new feature under development. ### How was this patch tested? New UT is added, also tested via DBeaver - the catalog/schema tree works now. Xnip2025-11-01_01-33-38 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52819 from pan3793/SPARK-54112. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 52fe51b6e9b366f4fdf108a5563a5a284c0bf4da) Signed-off-by: Dongjoon Hyun --- .../jdbc/SparkConnectDatabaseMetaData.scala | 67 ++++++++++- .../SparkConnectDatabaseMetaDataSuite.scala | 108 ++++++++++++++++++ .../spark/sql/connect/test/SQLHelper.scala | 12 ++ 3 files changed, 181 insertions(+), 6 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala index 215c8256acbc3..13dd4d57662b6 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala @@ -20,7 +20,11 @@ package org.apache.spark.sql.connect.client.jdbc import java.sql.{Array => _, _} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.util.QuotingUtils._ +import org.apache.spark.sql.connect import org.apache.spark.sql.connect.client.jdbc.SparkConnectDatabaseMetaData._ +import org.apache.spark.sql.functions._ import org.apache.spark.util.VersionUtils class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends DatabaseMetaData { @@ -97,8 +101,7 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas override def getTimeDateFunctions: String = throw new SQLFeatureNotSupportedException - override def getSearchStringEscape: String = - throw new SQLFeatureNotSupportedException + override def getSearchStringEscape: String = "\\" override def getExtraNameCharacters: String = "" @@ -277,6 +280,9 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas override def dataDefinitionIgnoredInTransactions: Boolean = false + private def isNullOrWildcard(pattern: String): Boolean = + pattern == null || pattern == "%" + override def getProcedures( catalog: String, schemaPattern: String, @@ -299,11 +305,60 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas new SparkConnectResultSet(df.collectResult()) } - override def getSchemas: ResultSet = - throw new SQLFeatureNotSupportedException + override def getSchemas: ResultSet = { + conn.checkOpen() - override def getSchemas(catalog: String, schemaPattern: String): ResultSet = - throw new SQLFeatureNotSupportedException + getSchemas(null, null) + } + + // Schema of the returned DataFrame is: + // |-- TABLE_SCHEM: string (nullable = false) + // |-- TABLE_CATALOG: string (nullable = false) + private def getSchemasDataFrame( + catalog: String, schemaPattern: String): connect.DataFrame = { + + val schemaFilterExpr = if (isNullOrWildcard(schemaPattern)) { + lit(true) + } else { + $"TABLE_SCHEM".like(schemaPattern) + } + + def internalGetSchemas( + catalogOpt: Option[String], + schemaFilterExpr: Column): connect.DataFrame = { + val catalog = catalogOpt.getOrElse(conn.getCatalog) + // Spark SQL supports LIKE clause in SHOW SCHEMAS command, but we can't use that + // because the LIKE pattern does not follow SQL standard. + conn.spark.sql(s"SHOW SCHEMAS IN ${quoteIdentifier(catalog)}") + .select($"namespace".as("TABLE_SCHEM")) + .filter(schemaFilterExpr) + .withColumn("TABLE_CATALOG", lit(catalog)) + } + + if (catalog == null) { + // search in all catalogs + val emptyDf = conn.spark.emptyDataFrame + .withColumn("TABLE_SCHEM", lit("")) + .withColumn("TABLE_CATALOG", lit("")) + conn.spark.catalog.listCatalogs().collect().map(_.name).map { c => + internalGetSchemas(Some(c), schemaFilterExpr) + }.fold(emptyDf) { (l, r) => l.unionAll(r) } + } else if (catalog == "") { + // search only in current catalog + internalGetSchemas(None, schemaFilterExpr) + } else { + // search in the specific catalog + internalGetSchemas(Some(catalog), schemaFilterExpr) + } + } + + override def getSchemas(catalog: String, schemaPattern: String): ResultSet = { + conn.checkOpen() + + val df = getSchemasDataFrame(catalog, schemaPattern) + .orderBy("TABLE_CATALOG", "TABLE_SCHEM") + new SparkConnectResultSet(df.collectResult()) + } override def getTableTypes: ResultSet = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala index 42596b56f4c56..c3d891bc38c38 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala @@ -69,6 +69,7 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark assert(metadata.storesLowerCaseQuotedIdentifiers === false) assert(metadata.storesMixedCaseQuotedIdentifiers === false) assert(metadata.getIdentifierQuoteString === "`") + assert(metadata.getSearchStringEscape === "\\") assert(metadata.getExtraNameCharacters === "") assert(metadata.supportsAlterTableWithAddColumn === true) assert(metadata.supportsAlterTableWithDropColumn === true) @@ -235,4 +236,111 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark } } } + + test("SparkConnectDatabaseMetaData getSchemas") { + + def verifyGetSchemas( + getSchemas: () => ResultSet)(verify: Seq[(String, String)] => Unit): Unit = { + Using.resource(getSchemas()) { rs => + val catalogDatabases = new Iterator[(String, String)] { + def hasNext: Boolean = rs.next() + def next(): (String, String) = + (rs.getString("TABLE_CATALOG"), rs.getString("TABLE_SCHEM")) + }.toSeq + verify(catalogDatabases) + } + } + + withConnection { conn => + implicit val spark: SparkSession = conn.asInstanceOf[SparkConnectConnection].spark + + registerCatalog("test`cat", TEST_IN_MEMORY_CATALOG) + + spark.sql("CREATE DATABASE IF NOT EXISTS `test``cat`.t_db1") + spark.sql("CREATE DATABASE IF NOT EXISTS `test``cat`.t_db2") + spark.sql("CREATE DATABASE IF NOT EXISTS `test``cat`.t_db_") + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db1") + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db2") + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.test_db3") + + val metadata = conn.getMetaData + + // no need to care about "test`cat" because it is memory based and session isolated, + // also is inaccessible from another SparkSession + withDatabase("spark_catalog.db1", "spark_catalog.db2", "spark_catalog.test_db3") { + // list schemas in all catalogs + val getSchemasInAllCatalogs = (() => metadata.getSchemas) :: + List(null, "%").map { database => () => metadata.getSchemas(null, database) } ::: Nil + + getSchemasInAllCatalogs.foreach { getSchemas => + verifyGetSchemas(getSchemas) { catalogDatabases => + // results are ordered by TABLE_CATALOG, TABLE_SCHEM + assert { + catalogDatabases === Seq( + ("spark_catalog", "db1"), + ("spark_catalog", "db2"), + ("spark_catalog", "default"), + ("spark_catalog", "test_db3"), + ("test`cat", "t_db1"), + ("test`cat", "t_db2"), + ("test`cat", "t_db_")) + } + } + } + + // list schemas in current catalog + assert(conn.getCatalog === "spark_catalog") + val getSchemasInCurrentCatalog = + List(null, "%").map { database => () => metadata.getSchemas("", database) } + getSchemasInCurrentCatalog.foreach { getSchemas => + verifyGetSchemas(getSchemas) { catalogDatabases => + // results are ordered by TABLE_CATALOG, TABLE_SCHEM + assert { + catalogDatabases === Seq( + ("spark_catalog", "db1"), + ("spark_catalog", "db2"), + ("spark_catalog", "default"), + ("spark_catalog", "test_db3")) + } + } + } + + // list schemas with schema pattern + verifyGetSchemas { () => metadata.getSchemas(null, "db%") } { catalogDatabases => + // results are ordered by TABLE_CATALOG, TABLE_SCHEM + assert { + catalogDatabases === Seq( + ("spark_catalog", "db1"), + ("spark_catalog", "db2")) + } + } + + verifyGetSchemas { () => metadata.getSchemas(null, "db_") } { catalogDatabases => + // results are ordered by TABLE_CATALOG, TABLE_SCHEM + assert { + catalogDatabases === Seq( + ("spark_catalog", "db1"), + ("spark_catalog", "db2")) + } + } + + // escape backtick in catalog, and _ in schema pattern + verifyGetSchemas { + () => metadata.getSchemas("test`cat", "t\\_db\\_") + } { catalogDatabases => + assert(catalogDatabases === Seq(("test`cat", "t_db_"))) + } + + // skip testing escape ', % in schema pattern, because Spark SQL does not + // allow using those chars in schema table name. + // + // CREATE DATABASE IF NOT EXISTS `t_db1'`; + // + // the above SQL fails with error condition: + // [INVALID_SCHEMA_OR_RELATION_NAME] `t_db1'` is not a valid name for tables/schemas. + // Valid names only contain alphabet characters, numbers and _. SQLSTATE: 42602 + } + } + } } diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/SQLHelper.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/SQLHelper.scala index b8d1062c3b3b0..731550363fc0a 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/SQLHelper.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/SQLHelper.scala @@ -142,4 +142,16 @@ trait SQLHelper { spark.sql(s"DROP VIEW IF EXISTS $name") }) } + + /** + * Drops database `dbName` after calling `f`. + */ + protected def withDatabase(dbNames: String*)(f: => Unit): Unit = { + SparkErrorUtils.tryWithSafeFinally(f) { + dbNames.foreach { name => + spark.sql(s"DROP DATABASE IF EXISTS $name CASCADE") + } + spark.sql(s"USE default") + } + } } From 8b4a9ad9c2dbcbfec39f9448b1a01074b768e75f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 10 Nov 2025 13:11:01 -0800 Subject: [PATCH 094/400] [SPARK-54287][PYTHON] Add Python 3.14 support in `pyspark-client` and `pyspark-connect` ### What changes were proposed in this pull request? This PR aims to add `Python 3.14` support in `pyspark-client` and `pyspark-connect` for Apache Spark 4.1.0. ### Why are the changes needed? Apache Spark 4.0+ has three Python packages. - `pyspark` contains the whole package to run PySpark, Spark Classic by default - `pyspark-connect` depends on `pyspark`, Spark Connect by default - `pyspark-client` only contains Python files to work as a Spark Connect client Like `pyspark` package, we need to add it consistently. - #52556 ### Does this PR introduce _any_ user-facing change? No behavior change because Python 3.14 support is newly added at Apache Spark 4.1.0. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52981 from dongjoon-hyun/SPARK-54287. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 67547efe6b7bb22d53de0de19c14448e26e183e4) Signed-off-by: Dongjoon Hyun --- python/packaging/client/setup.py | 1 + python/packaging/connect/setup.py | 1 + 2 files changed, 2 insertions(+) diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index c378d223cfcc3..b9dfce056a069 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -221,6 +221,7 @@ "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", "Programming Language :: Python :: 3.13", + "Programming Language :: Python :: 3.14", "Programming Language :: Python :: Implementation :: CPython", "Programming Language :: Python :: Implementation :: PyPy", "Typing :: Typed", diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index bd4d6fc93b6f6..03915f286cee3 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -131,6 +131,7 @@ "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", "Programming Language :: Python :: 3.13", + "Programming Language :: Python :: 3.14", "Programming Language :: Python :: Implementation :: CPython", "Programming Language :: Python :: Implementation :: PyPy", "Typing :: Typed", From 04b4716c40f66aefee62ba4f2dcaec7dc1f6b4ee Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Tue, 11 Nov 2025 09:11:04 -0400 Subject: [PATCH 095/400] [SPARK-54194][CONNECT] Spark Connect Proto Plan Compression MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Currently, Spark Connect enforce gRPC message limits on both the client and the server. These limits are largely meant to protect the server from potential OOMs by rejecting abnormally large messages. However, there are several cases where genuine messages exceed the limit and cause execution failures. To improve Spark Connect stability, this PR implements compressing unresolved proto plans to mitigate the issue of oversized messages from the client to the server. The compression applies to ExecutePlan and AnalyzePlan - the only two methods that might hit the message limit. The other issue of message limit from the server to the client is a different issue, and it’s out of the scope (that one is already fixed in https://github.com/apache/spark/pull/52271). In the implementation, * Zstandard is leveraged to compress proto plan as it has consistent high performance in our benchmark and achieves a good balance between compression ratio and performance. * The config `spark.connect.maxPlanSize` is introduced to control the maximum size of a (decompressed) proto plan that can be executed in Spark Connect. It is mainly used to avoid decompression bomb attacks. (Scala client changes are being implemented in a follow-up PR.) To reproduce the existing issue we are solving here, run this code on Spark Connect: ``` import random import string def random_letters(length: int) -> str: return ''.join(random.choices(string.ascii_letters, k=length)) num_unique_small_relations = 5 size_per_small_relation = 512 * 1024 small_dfs = [spark.createDataFrame([(random_letters(size_per_small_relation),)],) for _ in range(num_unique_small_relations)] result_df = small_dfs[0] for _ in range(512): result_df = result_df.unionByName(small_dfs[random.randint(0, len(small_dfs) - 1)]) result_df.collect() ``` It fails with `StatusCode.RESOURCE_EXHAUSTED` error with message`Sent message larger than max (269178955 vs. 134217728)`, because the client was trying to send a too large message to the server. Note: repeated small local relations is just one way causing a large plan, the size of the plan can also be contributed by repeated subtrees of plan transformations, serialized UDFs, captured external variables by UDFs, etc. With the improvement introduced by the PR, the above code runs successfully and prints the expected result. ### Why are the changes needed? It improves Spark Connect stability when executing and analyzing large plans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests on both the server side and the client side. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52894 from xi-db/plan-compression. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell (cherry picked from commit 0ccaacf54ce33db16f0b675e81a61b9b6a595479) Signed-off-by: Herman van Hovell --- .../resources/error/error-conditions.json | 22 + dev/requirements.txt | 1 + dev/spark-test-image/lint/Dockerfile | 1 + dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- .../python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- .../python-minimum/Dockerfile | 2 +- .../python-ps-minimum/Dockerfile | 2 +- python/packaging/classic/setup.py | 3 + python/packaging/client/setup.py | 2 + python/packaging/connect/setup.py | 2 + python/pyspark/sql/connect/client/core.py | 131 +++++- python/pyspark/sql/connect/plan.py | 3 +- python/pyspark/sql/connect/proto/base_pb2.py | 384 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 93 ++++- python/pyspark/sql/connect/utils.py | 17 + .../sql/tests/connect/test_connect_basic.py | 27 ++ .../sql/tests/connect/test_connect_plan.py | 2 +- python/pyspark/testing/connectutils.py | 13 + .../main/protobuf/spark/connect/base.proto | 26 +- .../spark/sql/connect/config/Connect.scala | 31 ++ .../ConnectPlanCompressionAlgorithm.scala | 21 + .../service/SparkConnectAnalyzeHandler.scala | 22 +- .../SparkConnectExecutePlanHandler.scala | 13 +- .../connect/utils/PlanCompressionUtils.scala | 118 ++++++ .../service/SparkConnectServiceE2ESuite.scala | 162 ++++++++ 30 files changed, 889 insertions(+), 223 deletions(-) create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index e061df065ed57..4f1292efc1a05 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -901,6 +901,28 @@ }, "sqlState" : "56K00" }, + "CONNECT_INVALID_PLAN" : { + "message" : [ + "The Spark Connect plan is invalid." + ], + "subClass" : { + "CANNOT_PARSE" : { + "message" : [ + "Cannot decompress or parse the input plan ()", + "This may be caused by a corrupted compressed plan.", + "To disable plan compression, set 'spark.connect.session.planCompression.threshold' to -1." + ] + }, + "PLAN_SIZE_LARGER_THAN_MAX" : { + "message" : [ + "The plan size is larger than max ( vs. )", + "This typically occurs when building very complex queries with many operations, large literals, or deeply nested expressions.", + "Consider splitting the query into smaller parts using temporary views for intermediate results or reducing the number of operations." + ] + } + }, + "sqlState" : "56K00" + }, "CONNECT_ML" : { "message" : [ "Generic Spark Connect ML error." diff --git a/dev/requirements.txt b/dev/requirements.txt index ddaeb9b3dd9db..cde0957715bfe 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -65,6 +65,7 @@ grpcio>=1.76.0 grpcio-status>=1.76.0 googleapis-common-protos>=1.71.0 protobuf==6.33.0 +zstandard>=0.25.0 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 4dfceae63a17a..6ab571bf35d6e 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -84,6 +84,7 @@ RUN python3.11 -m pip install \ 'grpc-stubs==1.24.11' \ 'grpcio-status==1.76.0' \ 'grpcio==1.76.0' \ + 'zstandard==0.25.0' \ 'ipython' \ 'ipython_genutils' \ 'jinja2' \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index bc9a507853c26..713e9e7d7ef4d 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=22.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index c318a615b7e05..9b5b18d061c2e 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 69d47e62774a5..f8a9df5842ce0 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 0c8b816f86297..ca62bc5ebc611 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index 1262089f43e14..b6e2dd7c80a97 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.13 packages diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 2e4dde33077da..bd64ecb31087d 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 07916fc35a0da..f3da21e005b30 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index 627dccdf34b11..575b4afdd02c0 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 13a5f2db386c8..5142d46cc3eb0 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index e6ac729f20d6f..54ec4abe3be91 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -156,6 +156,7 @@ def _supports_symlinks(): _minimum_grpc_version = "1.76.0" _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" +_minimum_zstandard_version = "0.25.0" class InstallCommand(install): @@ -366,6 +367,7 @@ def run(self): "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "numpy>=%s" % _minimum_numpy_version, ], "pipelines": [ @@ -375,6 +377,7 @@ def run(self): "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], }, diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index b9dfce056a069..ee404210f2932 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -139,6 +139,7 @@ _minimum_grpc_version = "1.76.0" _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" + _minimum_zstandard_version = "0.25.0" with open("README.md") as f: long_description = f.read() @@ -211,6 +212,7 @@ "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "numpy>=%s" % _minimum_numpy_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 03915f286cee3..9a1a4ea812555 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -92,6 +92,7 @@ _minimum_grpc_version = "1.76.0" _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" + _minimum_zstandard_version = "0.25.0" with open("README.md") as f: long_description = f.read() @@ -121,6 +122,7 @@ "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "numpy>=%s" % _minimum_numpy_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 414781d67cd45..44fa31474f590 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -34,6 +34,7 @@ import urllib.parse import uuid import sys +import time from typing import ( Iterable, Iterator, @@ -113,6 +114,19 @@ from pyspark.sql.datasource import DataSource +def _import_zstandard_if_available() -> Optional[Any]: + """ + Import zstandard if available, otherwise return None. + This is used to handle the case when zstandard is not installed. + """ + try: + import zstandard + + return zstandard + except ImportError: + return None + + class ChannelBuilder: """ This is a helper class that is used to create a GRPC channel based on the given @@ -706,6 +720,10 @@ def __init__( self._progress_handlers: List[ProgressHandler] = [] + self._zstd_module = _import_zstandard_if_available() + self._plan_compression_threshold: Optional[int] = None # Will be fetched lazily + self._plan_compression_algorithm: Optional[str] = None # Will be fetched lazily + # cleanup ml cache if possible atexit.register(self._cleanup_ml_cache) @@ -1156,7 +1174,7 @@ def execute_command( req = self._execute_plan_request_with_metadata() if self._user_id: req.user_context.user_id = self._user_id - req.plan.command.CopyFrom(command) + self._set_command_in_plan(req.plan, command) data, _, metrics, observed_metrics, properties = self._execute_and_fetch( req, observations or {} ) @@ -1182,7 +1200,7 @@ def execute_command_as_iterator( req = self._execute_plan_request_with_metadata() if self._user_id: req.user_context.user_id = self._user_id - req.plan.command.CopyFrom(command) + self._set_command_in_plan(req.plan, command) for response in self._execute_and_fetch_as_iterator(req, observations or {}): if isinstance(response, dict): yield response @@ -1963,6 +1981,17 @@ def _handle_rpc_error(self, rpc_error: grpc.RpcError) -> NoReturn: if info.metadata.get("errorClass") == "INVALID_HANDLE.SESSION_CHANGED": self._closed = True + if info.metadata.get("errorClass") == "CONNECT_INVALID_PLAN.CANNOT_PARSE": + # Disable plan compression if the server fails to interpret the plan. + logger.info( + "Disabling plan compression for the session due to " + "CONNECT_INVALID_PLAN.CANNOT_PARSE error." + ) + self._plan_compression_threshold, self._plan_compression_algorithm = ( + -1, + "NONE", + ) + raise convert_exception( info, status.message, @@ -2112,6 +2141,104 @@ def _query_model_size(self, model_ref_id: str) -> int: ml_command_result = properties["ml_command_result"] return ml_command_result.param.long + def _set_relation_in_plan(self, plan: pb2.Plan, relation: pb2.Relation) -> None: + """Sets the relation in the plan, attempting compression if configured.""" + self._try_compress_and_set_plan( + plan=plan, + message=relation, + op_type=pb2.Plan.CompressedOperation.OpType.OP_TYPE_RELATION, + ) + + def _set_command_in_plan(self, plan: pb2.Plan, command: pb2.Command) -> None: + """Sets the command in the plan, attempting compression if configured.""" + self._try_compress_and_set_plan( + plan=plan, + message=command, + op_type=pb2.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND, + ) + + def _try_compress_and_set_plan( + self, + plan: pb2.Plan, + message: google.protobuf.message.Message, + op_type: pb2.Plan.CompressedOperation.OpType.ValueType, + ) -> None: + """ + Tries to compress a protobuf message and sets it on the plan. + If compression is not enabled, not effective, or not available, + it falls back to the original message. + """ + ( + plan_compression_threshold, + plan_compression_algorithm, + ) = self._get_plan_compression_threshold_and_algorithm() + plan_compression_enabled = ( + plan_compression_threshold is not None + and plan_compression_threshold >= 0 + and plan_compression_algorithm is not None + and plan_compression_algorithm != "NONE" + ) + if plan_compression_enabled: + serialized_msg = message.SerializeToString() + original_size = len(serialized_msg) + if ( + original_size > plan_compression_threshold + and plan_compression_algorithm == "ZSTD" + and self._zstd_module + ): + start_time = time.time() + compressed_operation = pb2.Plan.CompressedOperation( + data=self._zstd_module.compress(serialized_msg), + op_type=op_type, + compression_codec=pb2.CompressionCodec.COMPRESSION_CODEC_ZSTD, + ) + duration = time.time() - start_time + compressed_size = len(compressed_operation.data) + logger.debug( + f"Plan compression: original_size={original_size}, " + f"compressed_size={compressed_size}, " + f"saving_ratio={1 - compressed_size / original_size:.2f}, " + f"duration_s={duration:.1f}" + ) + if compressed_size < original_size: + plan.compressed_operation.CopyFrom(compressed_operation) + return + else: + logger.debug("Plan compression not effective. Using original plan.") + + if op_type == pb2.Plan.CompressedOperation.OpType.OP_TYPE_RELATION: + plan.root.CopyFrom(message) # type: ignore[arg-type] + else: + plan.command.CopyFrom(message) # type: ignore[arg-type] + + def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: + if self._plan_compression_threshold is None or self._plan_compression_algorithm is None: + try: + ( + plan_compression_threshold_str, + self._plan_compression_algorithm, + ) = self.get_configs( + "spark.connect.session.planCompression.threshold", + "spark.connect.session.planCompression.defaultAlgorithm", + ) + self._plan_compression_threshold = ( + int(plan_compression_threshold_str) if plan_compression_threshold_str else -1 + ) + logger.debug( + f"Plan compression threshold: {self._plan_compression_threshold}, " + f"algorithm: {self._plan_compression_algorithm}" + ) + except Exception as e: + self._plan_compression_threshold = -1 + self._plan_compression_algorithm = "NONE" + logger.debug( + "Plan compression is disabled because the server does not support it.", e + ) + return ( + self._plan_compression_threshold, + self._plan_compression_algorithm, + ) # type: ignore[return-value] + def clone(self, new_session_id: Optional[str] = None) -> "SparkConnectClient": """ Clone this client session on the server side. The server-side session is cloned with diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 82a6326c7dc58..02fe7176b6fee 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -143,7 +143,8 @@ def to_proto(self, session: "SparkConnectClient", debug: bool = False) -> proto. if enabled, the proto plan will be printed. """ plan = proto.Plan() - plan.root.CopyFrom(self.plan(session)) + relation = self.plan(session) + session._set_relation_in_plan(plan, relation) if debug: print(plan) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 32bf6802df7bc..32b2840dffadc 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -45,7 +45,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto\x1a\x16spark/connect/ml.proto\x1a\x1dspark/connect/pipelines.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\x83\x06\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\x85\x02\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12^\n\x17result_chunking_options\x18\x02 \x01(\x0b\x32$.spark.connect.ResultChunkingOptionsH\x00R\x15resultChunkingOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\x81\x1b\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12L\n\x11ml_command_result\x18\x14 \x01(\x0b\x32\x1e.spark.connect.MlCommandResultH\x00R\x0fmlCommandResult\x12X\n\x15pipeline_event_result\x18\x15 \x01(\x0b\x32".spark.connect.PipelineEventResultH\x00R\x13pipelineEventResult\x12^\n\x17pipeline_command_result\x18\x16 \x01(\x0b\x32$.spark.connect.PipelineCommandResultH\x00R\x15pipelineCommandResult\x12\x8d\x01\n(pipeline_query_function_execution_signal\x18\x17 \x01(\x0b\x32\x33.spark.connect.PipelineQueryFunctionExecutionSignalH\x00R$pipelineQueryFunctionExecutionSignal\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a\xf8\x01\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x12$\n\x0b\x63hunk_index\x18\x04 \x01(\x03H\x01R\nchunkIndex\x88\x01\x01\x12\x32\n\x13num_chunks_in_batch\x18\x05 \x01(\x03H\x02R\x10numChunksInBatch\x88\x01\x01\x42\x0f\n\r_start_offsetB\x0e\n\x0c_chunk_indexB\x16\n\x14_num_chunks_in_batch\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\xb5\x01\n\x15ResultChunkingOptions\x12;\n\x1a\x61llow_arrow_batch_chunking\x18\x01 \x01(\x08R\x17\x61llowArrowBatchChunking\x12@\n\x1apreferred_arrow_chunk_size\x18\x02 \x01(\x03H\x00R\x17preferredArrowChunkSize\x88\x01\x01\x42\x1d\n\x1b_preferred_arrow_chunk_size"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xd9\x0f\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\xa6\x04\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x12r\n\x14\x62reaking_change_info\x18\x05 \x01(\x0b\x32;.spark.connect.FetchErrorDetailsResponse.BreakingChangeInfoH\x02R\x12\x62reakingChangeInfo\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_stateB\x17\n\x15_breaking_change_info\x1a\xfa\x01\n\x12\x42reakingChangeInfo\x12+\n\x11migration_message\x18\x01 \x03(\tR\x10migrationMessage\x12k\n\x11mitigation_config\x18\x02 \x01(\x0b\x32\x39.spark.connect.FetchErrorDetailsResponse.MitigationConfigH\x00R\x10mitigationConfig\x88\x01\x01\x12$\n\x0bneeds_audit\x18\x03 \x01(\x08H\x01R\nneedsAudit\x88\x01\x01\x42\x14\n\x12_mitigation_configB\x0e\n\x0c_needs_audit\x1a:\n\x10MitigationConfig\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xea\x02\n\x13\x43loneSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12)\n\x0enew_session_id\x18\x04 \x01(\tH\x02R\x0cnewSessionId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x11\n\x0f_new_session_id"\xcc\x01\n\x14\x43loneSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12$\n\x0enew_session_id\x18\x03 \x01(\tR\x0cnewSessionId\x12:\n\x1anew_server_side_session_id\x18\x04 \x01(\tR\x16newServerSideSessionId2\x8d\x08\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x12Y\n\x0c\x43loneSession\x12".spark.connect.CloneSessionRequest\x1a#.spark.connect.CloneSessionResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto\x1a\x16spark/connect/ml.proto\x1a\x1dspark/connect/pipelines.proto"\xe3\x03\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommand\x12\\\n\x14\x63ompressed_operation\x18\x03 \x01(\x0b\x32\'.spark.connect.Plan.CompressedOperationH\x00R\x13\x63ompressedOperation\x1a\x8e\x02\n\x13\x43ompressedOperation\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12G\n\x07op_type\x18\x02 \x01(\x0e\x32..spark.connect.Plan.CompressedOperation.OpTypeR\x06opType\x12L\n\x11\x63ompression_codec\x18\x03 \x01(\x0e\x32\x1f.spark.connect.CompressionCodecR\x10\x63ompressionCodec"L\n\x06OpType\x12\x17\n\x13OP_TYPE_UNSPECIFIED\x10\x00\x12\x14\n\x10OP_TYPE_RELATION\x10\x01\x12\x13\n\x0fOP_TYPE_COMMAND\x10\x02\x42\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\x83\x06\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\x85\x02\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12^\n\x17result_chunking_options\x18\x02 \x01(\x0b\x32$.spark.connect.ResultChunkingOptionsH\x00R\x15resultChunkingOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\x81\x1b\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12L\n\x11ml_command_result\x18\x14 \x01(\x0b\x32\x1e.spark.connect.MlCommandResultH\x00R\x0fmlCommandResult\x12X\n\x15pipeline_event_result\x18\x15 \x01(\x0b\x32".spark.connect.PipelineEventResultH\x00R\x13pipelineEventResult\x12^\n\x17pipeline_command_result\x18\x16 \x01(\x0b\x32$.spark.connect.PipelineCommandResultH\x00R\x15pipelineCommandResult\x12\x8d\x01\n(pipeline_query_function_execution_signal\x18\x17 \x01(\x0b\x32\x33.spark.connect.PipelineQueryFunctionExecutionSignalH\x00R$pipelineQueryFunctionExecutionSignal\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a\xf8\x01\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x12$\n\x0b\x63hunk_index\x18\x04 \x01(\x03H\x01R\nchunkIndex\x88\x01\x01\x12\x32\n\x13num_chunks_in_batch\x18\x05 \x01(\x03H\x02R\x10numChunksInBatch\x88\x01\x01\x42\x0f\n\r_start_offsetB\x0e\n\x0c_chunk_indexB\x16\n\x14_num_chunks_in_batch\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\xb5\x01\n\x15ResultChunkingOptions\x12;\n\x1a\x61llow_arrow_batch_chunking\x18\x01 \x01(\x08R\x17\x61llowArrowBatchChunking\x12@\n\x1apreferred_arrow_chunk_size\x18\x02 \x01(\x03H\x00R\x17preferredArrowChunkSize\x88\x01\x01\x42\x1d\n\x1b_preferred_arrow_chunk_size"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xd9\x0f\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\xa6\x04\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x12r\n\x14\x62reaking_change_info\x18\x05 \x01(\x0b\x32;.spark.connect.FetchErrorDetailsResponse.BreakingChangeInfoH\x02R\x12\x62reakingChangeInfo\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_stateB\x17\n\x15_breaking_change_info\x1a\xfa\x01\n\x12\x42reakingChangeInfo\x12+\n\x11migration_message\x18\x01 \x03(\tR\x10migrationMessage\x12k\n\x11mitigation_config\x18\x02 \x01(\x0b\x32\x39.spark.connect.FetchErrorDetailsResponse.MitigationConfigH\x00R\x10mitigationConfig\x88\x01\x01\x12$\n\x0bneeds_audit\x18\x03 \x01(\x08H\x01R\nneedsAudit\x88\x01\x01\x42\x14\n\x12_mitigation_configB\x0e\n\x0c_needs_audit\x1a:\n\x10MitigationConfig\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xea\x02\n\x13\x43loneSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12)\n\x0enew_session_id\x18\x04 \x01(\tH\x02R\x0cnewSessionId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x11\n\x0f_new_session_id"\xcc\x01\n\x14\x43loneSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12$\n\x0enew_session_id\x18\x03 \x01(\tR\x0cnewSessionId\x12:\n\x1anew_server_side_session_id\x18\x04 \x01(\tR\x16newServerSideSessionId*Q\n\x10\x43ompressionCodec\x12!\n\x1d\x43OMPRESSION_CODEC_UNSPECIFIED\x10\x00\x12\x1a\n\x16\x43OMPRESSION_CODEC_ZSTD\x10\x01\x32\x8d\x08\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x12Y\n\x0c\x43loneSession\x12".spark.connect.CloneSessionRequest\x1a#.spark.connect.CloneSessionResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -70,200 +70,206 @@ _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" ]._serialized_options = b"8\001" - _globals["_PLAN"]._serialized_start = 274 - _globals["_PLAN"]._serialized_end = 390 - _globals["_USERCONTEXT"]._serialized_start = 392 - _globals["_USERCONTEXT"]._serialized_end = 514 - _globals["_ANALYZEPLANREQUEST"]._serialized_start = 517 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3194 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1879 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1928 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1931 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2246 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2074 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2246 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2248 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2338 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2340 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2390 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2392 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2446 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2448 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2501 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2503 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2517 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2519 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2560 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2562 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2683 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2685 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2740 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2743 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2894 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2896 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 3006 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 3008 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3078 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3080 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3124 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3197 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5063 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4438 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4495 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4497 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4545 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4547 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4592 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4594 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4630 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4632 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4680 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4682 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4716 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4718 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4758 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4760 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4819 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4821 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4860 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4862 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4900 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2743 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2752 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2896 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2907 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4926 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 5009 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 5011 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5053 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5066 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5837 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5500 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5761 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5840 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 9297 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7940 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8011 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8014 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8262 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8265 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8782 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 8360 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8692 + _globals["_COMPRESSIONCODEC"]._serialized_start = 18571 + _globals["_COMPRESSIONCODEC"]._serialized_end = 18652 + _globals["_PLAN"]._serialized_start = 275 + _globals["_PLAN"]._serialized_end = 758 + _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_start = 477 + _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_end = 747 + _globals["_PLAN_COMPRESSEDOPERATION_OPTYPE"]._serialized_start = 671 + _globals["_PLAN_COMPRESSEDOPERATION_OPTYPE"]._serialized_end = 747 + _globals["_USERCONTEXT"]._serialized_start = 760 + _globals["_USERCONTEXT"]._serialized_end = 882 + _globals["_ANALYZEPLANREQUEST"]._serialized_start = 885 + _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3562 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 2247 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 2296 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 2299 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2614 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2442 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2614 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2616 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2706 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2708 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2758 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2760 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2814 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2816 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2869 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2871 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2885 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2887 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2928 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2930 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 3051 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 3053 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 3108 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 3111 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 3262 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 3264 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 3374 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 3376 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3446 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3448 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3492 + _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3565 + _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5431 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4806 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4863 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4865 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4913 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4915 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4960 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4962 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4998 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 5000 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 5048 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 5050 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 5084 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 5086 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 5126 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 5128 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 5187 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 5189 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 5228 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 5230 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 5268 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 3111 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 3120 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 3264 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 3275 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 5294 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 5377 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 5379 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5421 + _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5434 + _globals["_EXECUTEPLANREQUEST"]._serialized_end = 6205 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5868 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 6129 + _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 6208 + _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 9665 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 8308 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8379 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8382 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8630 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8633 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 9150 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 8728 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 9060 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 8569 + ]._serialized_start = 8937 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 8692 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8694 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8782 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8785 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8926 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8928 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8944 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8947 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9280 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9103 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9280 - _globals["_KEYVALUE"]._serialized_start = 9299 - _globals["_KEYVALUE"]._serialized_end = 9364 - _globals["_CONFIGREQUEST"]._serialized_start = 9367 - _globals["_CONFIGREQUEST"]._serialized_end = 10566 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 9675 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10173 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 10175 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 10267 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 10269 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 10294 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10296 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10359 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10361 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10392 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10394 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10442 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10444 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10471 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10473 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10507 - _globals["_CONFIGRESPONSE"]._serialized_start = 10569 - _globals["_CONFIGRESPONSE"]._serialized_end = 10744 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10747 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11749 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11222 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11275 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11277 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11388 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11390 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11483 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11486 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 11679 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11752 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12024 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11943 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12024 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12027 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12353 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12356 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12708 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12551 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 12666 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 12668 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12708 - _globals["_INTERRUPTREQUEST"]._serialized_start = 12711 - _globals["_INTERRUPTREQUEST"]._serialized_end = 13314 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13114 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13242 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 13317 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 13461 - _globals["_REATTACHOPTIONS"]._serialized_start = 13463 - _globals["_REATTACHOPTIONS"]._serialized_end = 13516 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13519 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 13700 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 13703 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14109 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14112 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 14697 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14566 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14578 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14580 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14627 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 14700 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 14865 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 14868 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15080 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15082 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15190 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15193 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15525 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15528 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17537 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 15757 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15931 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15934 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16302 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16265 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16302 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16305 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 16855 + ]._serialized_end = 9060 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 9062 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 9150 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 9153 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 9294 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 9296 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 9312 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 9315 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9648 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9471 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9648 + _globals["_KEYVALUE"]._serialized_start = 9667 + _globals["_KEYVALUE"]._serialized_end = 9732 + _globals["_CONFIGREQUEST"]._serialized_start = 9735 + _globals["_CONFIGREQUEST"]._serialized_end = 10934 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 10043 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10541 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 10543 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 10635 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 10637 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 10662 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10664 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10727 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10729 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10760 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10762 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10810 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10812 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10839 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10841 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10875 + _globals["_CONFIGRESPONSE"]._serialized_start = 10937 + _globals["_CONFIGRESPONSE"]._serialized_end = 11112 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 11115 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 12117 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11590 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11643 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11645 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11756 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11758 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11851 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11854 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 12047 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 12120 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12392 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 12311 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12392 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12395 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12721 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12724 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 13076 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12919 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 13034 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 13036 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 13076 + _globals["_INTERRUPTREQUEST"]._serialized_start = 13079 + _globals["_INTERRUPTREQUEST"]._serialized_end = 13682 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13482 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13610 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 13685 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 13829 + _globals["_REATTACHOPTIONS"]._serialized_start = 13831 + _globals["_REATTACHOPTIONS"]._serialized_end = 13884 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13887 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 14068 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 14071 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14477 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14480 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 15065 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14934 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14946 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14948 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14995 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 15068 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 15233 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 15236 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15448 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15450 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15558 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15561 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15893 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15896 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17905 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 16125 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 16299 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 16302 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16670 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16633 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16670 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16673 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 17223 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 16732 + ]._serialized_start = 17100 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 16800 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 16858 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17108 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17110 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17168 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17171 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17518 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17539 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17629 - _globals["_CLONESESSIONREQUEST"]._serialized_start = 17632 - _globals["_CLONESESSIONREQUEST"]._serialized_end = 17994 - _globals["_CLONESESSIONRESPONSE"]._serialized_start = 17997 - _globals["_CLONESESSIONRESPONSE"]._serialized_end = 18201 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 18204 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 19241 + ]._serialized_end = 17168 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 17226 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17476 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17478 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17536 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17539 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17886 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17907 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17997 + _globals["_CLONESESSIONREQUEST"]._serialized_start = 18000 + _globals["_CLONESESSIONREQUEST"]._serialized_end = 18362 + _globals["_CLONESESSIONRESPONSE"]._serialized_start = 18365 + _globals["_CLONESESSIONRESPONSE"]._serialized_end = 18569 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 18655 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 19692 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index dc3099ecdffca..f12c21e5536de 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -57,42 +57,123 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +class _CompressionCodec: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + +class _CompressionCodecEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[_CompressionCodec.ValueType], + builtins.type, +): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + COMPRESSION_CODEC_UNSPECIFIED: _CompressionCodec.ValueType # 0 + COMPRESSION_CODEC_ZSTD: _CompressionCodec.ValueType # 1 + +class CompressionCodec(_CompressionCodec, metaclass=_CompressionCodecEnumTypeWrapper): + """Compression codec for plan compression.""" + +COMPRESSION_CODEC_UNSPECIFIED: CompressionCodec.ValueType # 0 +COMPRESSION_CODEC_ZSTD: CompressionCodec.ValueType # 1 +global___CompressionCodec = CompressionCodec + class Plan(google.protobuf.message.Message): """A [[Plan]] is the structure that carries the runtime information for the execution from the - client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference - to the underlying logical plan or it can be of the [[Command]] type that is used to execute - commands on the server. + client to the server. A [[Plan]] can be one of the following: + - [[Relation]]: a reference to the underlying logical plan. + - [[Command]]: used to execute commands on the server. + - [[CompressedOperation]]: a compressed representation of either a Relation or a Command. """ DESCRIPTOR: google.protobuf.descriptor.Descriptor + class CompressedOperation(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _OpType: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _OpTypeEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + Plan.CompressedOperation._OpType.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + OP_TYPE_UNSPECIFIED: Plan.CompressedOperation._OpType.ValueType # 0 + OP_TYPE_RELATION: Plan.CompressedOperation._OpType.ValueType # 1 + OP_TYPE_COMMAND: Plan.CompressedOperation._OpType.ValueType # 2 + + class OpType(_OpType, metaclass=_OpTypeEnumTypeWrapper): ... + OP_TYPE_UNSPECIFIED: Plan.CompressedOperation.OpType.ValueType # 0 + OP_TYPE_RELATION: Plan.CompressedOperation.OpType.ValueType # 1 + OP_TYPE_COMMAND: Plan.CompressedOperation.OpType.ValueType # 2 + + DATA_FIELD_NUMBER: builtins.int + OP_TYPE_FIELD_NUMBER: builtins.int + COMPRESSION_CODEC_FIELD_NUMBER: builtins.int + data: builtins.bytes + op_type: global___Plan.CompressedOperation.OpType.ValueType + compression_codec: global___CompressionCodec.ValueType + def __init__( + self, + *, + data: builtins.bytes = ..., + op_type: global___Plan.CompressedOperation.OpType.ValueType = ..., + compression_codec: global___CompressionCodec.ValueType = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "compression_codec", b"compression_codec", "data", b"data", "op_type", b"op_type" + ], + ) -> None: ... + ROOT_FIELD_NUMBER: builtins.int COMMAND_FIELD_NUMBER: builtins.int + COMPRESSED_OPERATION_FIELD_NUMBER: builtins.int @property def root(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: ... @property def command(self) -> pyspark.sql.connect.proto.commands_pb2.Command: ... + @property + def compressed_operation(self) -> global___Plan.CompressedOperation: ... def __init__( self, *, root: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., command: pyspark.sql.connect.proto.commands_pb2.Command | None = ..., + compressed_operation: global___Plan.CompressedOperation | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "command", b"command", "op_type", b"op_type", "root", b"root" + "command", + b"command", + "compressed_operation", + b"compressed_operation", + "op_type", + b"op_type", + "root", + b"root", ], ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "command", b"command", "op_type", b"op_type", "root", b"root" + "command", + b"command", + "compressed_operation", + b"compressed_operation", + "op_type", + b"op_type", + "root", + b"root", ], ) -> None: ... def WhichOneof( self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] - ) -> typing_extensions.Literal["root", "command"] | None: ... + ) -> typing_extensions.Literal["root", "command", "compressed_operation"] | None: ... global___Plan = Plan diff --git a/python/pyspark/sql/connect/utils.py b/python/pyspark/sql/connect/utils.py index a2511836816c9..0e0e042446531 100644 --- a/python/pyspark/sql/connect/utils.py +++ b/python/pyspark/sql/connect/utils.py @@ -37,6 +37,7 @@ def check_dependencies(mod_name: str) -> None: require_minimum_grpc_version() require_minimum_grpcio_status_version() require_minimum_googleapis_common_protos_version() + require_minimum_zstandard_version() def require_minimum_grpc_version() -> None: @@ -96,5 +97,21 @@ def require_minimum_googleapis_common_protos_version() -> None: ) from error +def require_minimum_zstandard_version() -> None: + """Raise ImportError if zstandard is not installed""" + minimum_zstandard_version = "0.25.0" + + try: + import zstandard # noqa + except ImportError as error: + raise PySparkImportError( + errorClass="PACKAGE_NOT_INSTALLED", + messageParameters={ + "package_name": "zstandard", + "minimum_version": str(minimum_zstandard_version), + }, + ) from error + + def get_python_ver() -> str: return "%d.%d" % sys.version_info[:2] diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index c1ba9a6fc2d4a..b789d7919c94e 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -1447,6 +1447,33 @@ def test_truncate_message(self): proto_string_truncated_3 = self.connect._client._proto_to_string(plan3, True) self.assertTrue(len(proto_string_truncated_3) < 64000, len(proto_string_truncated_3)) + def test_plan_compression(self): + self.assertTrue(self.connect._client._zstd_module is not None) + self.connect.range(1).count() + default_plan_compression_threshold = self.connect._client._plan_compression_threshold + self.assertTrue(default_plan_compression_threshold > 0) + self.assertTrue(self.connect._client._plan_compression_algorithm == "ZSTD") + try: + self.connect._client._plan_compression_threshold = 1000 + + # Small plan should not be compressed + cdf1 = self.connect.range(1).select(CF.lit("Apache Spark")) + plan1 = cdf1._plan.to_proto(self.connect._client) + self.assertTrue(plan1.root is not None) + self.assertTrue(cdf1.count() == 1) + + # Large plan should be compressed + cdf2 = self.connect.range(1).select(CF.lit("Apache Spark" * 1000)) + plan2 = cdf2._plan.to_proto(self.connect._client) + self.assertTrue(plan2.compressed_operation is not None) + # Test compressed relation + self.assertTrue(cdf2.count() == 1) + # Test compressed command + cdf2.createOrReplaceTempView("temp_view_cdf2") + self.assertTrue(self.connect.sql("SELECT * FROM temp_view_cdf2").count() == 1) + finally: + self.connect._client._plan_compression_threshold = default_plan_compression_threshold + class SparkConnectGCTests(SparkConnectSQLTestCase): @classmethod diff --git a/python/pyspark/sql/tests/connect/test_connect_plan.py b/python/pyspark/sql/tests/connect/test_connect_plan.py index d25799f0c9f26..1d4d85e8426ee 100644 --- a/python/pyspark/sql/tests/connect/test_connect_plan.py +++ b/python/pyspark/sql/tests/connect/test_connect_plan.py @@ -864,7 +864,7 @@ def test_uuid_literal(self): def test_column_literals(self): df = self.connect.with_plan(Read("table")) lit_df = df.select(lit(10)) - self.assertIsNotNone(lit_df._plan.to_proto(None)) + self.assertIsNotNone(lit_df._plan.to_proto(self.connect)) self.assertIsNotNone(lit(10).to_plan(None)) plan = lit(10).to_plan(None) diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index bfcb886e1c912..d895c1d8a26b1 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -23,6 +23,7 @@ import uuid import contextlib +import pyspark.sql.connect.proto as pb2 from pyspark import Row, SparkConf from pyspark.util import is_remote_only from pyspark.testing.utils import PySparkErrorTestUtils @@ -113,6 +114,16 @@ def _session_range( def _session_sql(cls, query): return cls._df_mock(SQL(query)) + @classmethod + def _set_relation_in_plan(self, plan: pb2.Plan, relation: pb2.Relation) -> None: + # Skip plan compression in plan-only tests. + plan.root.CopyFrom(relation) + + @classmethod + def _set_command_in_plan(self, plan: pb2.Plan, command: pb2.Command) -> None: + # Skip plan compression in plan-only tests. + plan.command.CopyFrom(command) + if have_pandas: @classmethod @@ -129,6 +140,8 @@ def setUpClass(cls): cls.connect.set_hook("range", cls._session_range) cls.connect.set_hook("sql", cls._session_sql) cls.connect.set_hook("with_plan", cls._with_plan) + cls.connect.set_hook("_set_relation_in_plan", cls._set_relation_in_plan) + cls.connect.set_hook("_set_command_in_plan", cls._set_command_in_plan) @classmethod def tearDownClass(cls): diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index 6e1029bf0a6a2..a97d2d25f490e 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -33,17 +33,35 @@ option java_package = "org.apache.spark.connect.proto"; option go_package = "internal/generated"; // A [[Plan]] is the structure that carries the runtime information for the execution from the -// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference -// to the underlying logical plan or it can be of the [[Command]] type that is used to execute -// commands on the server. +// client to the server. A [[Plan]] can be one of the following: +// - [[Relation]]: a reference to the underlying logical plan. +// - [[Command]]: used to execute commands on the server. +// - [[CompressedOperation]]: a compressed representation of either a Relation or a Command. message Plan { oneof op_type { Relation root = 1; Command command = 2; + CompressedOperation compressed_operation = 3; } -} + message CompressedOperation { + bytes data = 1; + OpType op_type = 2; + CompressionCodec compression_codec = 3; + enum OpType { + OP_TYPE_UNSPECIFIED = 0; + OP_TYPE_RELATION = 1; + OP_TYPE_COMMAND = 2; + } + } +} + +// Compression codec for plan compression. +enum CompressionCodec { + COMPRESSION_CODEC_UNSPECIFIED = 0; + COMPRESSION_CODEC_ZSTD = 1; +} // User Context is used to refer to one particular user session that is executing // queries in the backend. diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index c6049187f6be8..1ffed714b4caa 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.connect.config +import java.util.Locale import java.util.concurrent.TimeUnit import org.apache.spark.SparkEnv @@ -418,4 +419,34 @@ object Connect { .bytesConf(ByteUnit.BYTE) // 90% of the max message size by default to allow for some overhead. .createWithDefault((ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE * 0.9).toInt) + + private[spark] val CONNECT_MAX_PLAN_SIZE = + buildStaticConf("spark.connect.maxPlanSize") + .doc( + "The maximum size of a (decompressed) proto plan that can be executed in Spark " + + "Connect. If the size of the plan exceeds this limit, an error will be thrown. " + + "The size is in bytes.") + .version("4.1.0") + .internal() + .bytesConf(ByteUnit.BYTE) + .createWithDefault(512 * 1024 * 1024) // 512 MB + + val CONNECT_SESSION_PLAN_COMPRESSION_THRESHOLD = + buildConf("spark.connect.session.planCompression.threshold") + .doc("The threshold in bytes for the size of proto plan to be compressed. " + + "If the size of proto plan is smaller than this threshold, it will not be compressed.") + .version("4.1.0") + .internal() + .intConf + .createWithDefault(10 * 1024 * 1024) // 10 MB + + val CONNECT_PLAN_COMPRESSION_DEFAULT_ALGORITHM = + buildConf("spark.connect.session.planCompression.defaultAlgorithm") + .doc("The default algorithm of proto plan compression.") + .version("4.1.0") + .internal() + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValues(ConnectPlanCompressionAlgorithm.values.map(_.toString)) + .createWithDefault(ConnectPlanCompressionAlgorithm.ZSTD.toString) } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala new file mode 100644 index 0000000000000..0f9b959ee7256 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect.config + +object ConnectPlanCompressionAlgorithm extends Enumeration { + val ZSTD = Value +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala index 8fa003c11681d..cdf7013211f77 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.classic.{DataFrame, Dataset} import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput, StorageLevelProtoConverter} import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.sql.connect.utils.PlanCompressionUtils import org.apache.spark.sql.execution.{CodegenMode, CommandExecutionMode, CostMode, ExtendedMode, FormattedMode, SimpleMode} import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -63,6 +64,9 @@ private[connect] class SparkConnectAnalyzeHandler( val builder = proto.AnalyzePlanResponse.newBuilder() def transformRelation(rel: proto.Relation) = planner.transformRelation(rel, cachePlan = true) + def transformRelationPlan(plan: proto.Plan) = { + transformRelation(PlanCompressionUtils.decompressPlan(plan).getRoot) + } def getDataFrameWithoutExecuting(rel: LogicalPlan): DataFrame = { val qe = session.sessionState.executePlan(rel, CommandExecutionMode.SKIP) @@ -71,7 +75,7 @@ private[connect] class SparkConnectAnalyzeHandler( request.getAnalyzeCase match { case proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA => - val rel = transformRelation(request.getSchema.getPlan.getRoot) + val rel = transformRelationPlan(request.getSchema.getPlan) val schema = getDataFrameWithoutExecuting(rel).schema builder.setSchema( proto.AnalyzePlanResponse.Schema @@ -79,7 +83,7 @@ private[connect] class SparkConnectAnalyzeHandler( .setSchema(DataTypeProtoConverter.toConnectProtoType(schema)) .build()) case proto.AnalyzePlanRequest.AnalyzeCase.EXPLAIN => - val rel = transformRelation(request.getExplain.getPlan.getRoot) + val rel = transformRelationPlan(request.getExplain.getPlan) val queryExecution = getDataFrameWithoutExecuting(rel).queryExecution val explainString = request.getExplain.getExplainMode match { case proto.AnalyzePlanRequest.Explain.ExplainMode.EXPLAIN_MODE_SIMPLE => @@ -101,7 +105,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.TREE_STRING => - val rel = transformRelation(request.getTreeString.getPlan.getRoot) + val rel = transformRelationPlan(request.getTreeString.getPlan) val schema = getDataFrameWithoutExecuting(rel).schema val treeString = if (request.getTreeString.hasLevel) { schema.treeString(request.getTreeString.getLevel) @@ -115,7 +119,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.IS_LOCAL => - val rel = transformRelation(request.getIsLocal.getPlan.getRoot) + val rel = transformRelationPlan(request.getIsLocal.getPlan) val isLocal = getDataFrameWithoutExecuting(rel).isLocal builder.setIsLocal( proto.AnalyzePlanResponse.IsLocal @@ -124,7 +128,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.IS_STREAMING => - val rel = transformRelation(request.getIsStreaming.getPlan.getRoot) + val rel = transformRelationPlan(request.getIsStreaming.getPlan) val isStreaming = getDataFrameWithoutExecuting(rel).isStreaming builder.setIsStreaming( proto.AnalyzePlanResponse.IsStreaming @@ -133,7 +137,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.INPUT_FILES => - val rel = transformRelation(request.getInputFiles.getPlan.getRoot) + val rel = transformRelationPlan(request.getInputFiles.getPlan) val inputFiles = getDataFrameWithoutExecuting(rel).inputFiles builder.setInputFiles( proto.AnalyzePlanResponse.InputFiles @@ -157,8 +161,8 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.SAME_SEMANTICS => - val targetRel = transformRelation(request.getSameSemantics.getTargetPlan.getRoot) - val otherRel = transformRelation(request.getSameSemantics.getOtherPlan.getRoot) + val targetRel = transformRelationPlan(request.getSameSemantics.getTargetPlan) + val otherRel = transformRelationPlan(request.getSameSemantics.getOtherPlan) val target = getDataFrameWithoutExecuting(targetRel) val other = getDataFrameWithoutExecuting(otherRel) builder.setSameSemantics( @@ -167,7 +171,7 @@ private[connect] class SparkConnectAnalyzeHandler( .setResult(target.sameSemantics(other))) case proto.AnalyzePlanRequest.AnalyzeCase.SEMANTIC_HASH => - val rel = transformRelation(request.getSemanticHash.getPlan.getRoot) + val rel = transformRelationPlan(request.getSemanticHash.getPlan) val semanticHash = getDataFrameWithoutExecuting(rel) .semanticHash() builder.setSemanticHash( diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala index 027f4517cf3be..6780ca37e96a7 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala @@ -22,6 +22,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.SparkSQLException import org.apache.spark.connect.proto import org.apache.spark.internal.Logging +import org.apache.spark.sql.connect.utils.PlanCompressionUtils class SparkConnectExecutePlanHandler(responseObserver: StreamObserver[proto.ExecutePlanResponse]) extends Logging { @@ -35,12 +36,20 @@ class SparkConnectExecutePlanHandler(responseObserver: StreamObserver[proto.Exec .getOrCreateIsolatedSession(v.getUserContext.getUserId, v.getSessionId, previousSessionId) val executeKey = ExecuteKey(v, sessionHolder) + val decompressedRequest = + v.toBuilder.setPlan(PlanCompressionUtils.decompressPlan(v.getPlan)).build() + SparkConnectService.executionManager.getExecuteHolder(executeKey) match { case None => // Create a new execute holder and attach to it. SparkConnectService.executionManager - .createExecuteHolderAndAttach(executeKey, v, sessionHolder, responseObserver) - case Some(executeHolder) if executeHolder.request.getPlan.equals(v.getPlan) => + .createExecuteHolderAndAttach( + executeKey, + decompressedRequest, + sessionHolder, + responseObserver) + case Some(executeHolder) + if executeHolder.request.getPlan.equals(decompressedRequest.getPlan) => // If the execute holder already exists with the same plan, reattach to it. SparkConnectService.executionManager .reattachExecuteHolder(executeHolder, responseObserver, None) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala new file mode 100644 index 0000000000000..708ef1ee6558f --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.utils + +import java.io.IOException + +import scala.util.control.NonFatal + +import com.github.luben.zstd.{Zstd, ZstdInputStreamNoFinalizer} +import com.google.protobuf.{ByteString, CodedInputStream} +import org.apache.commons.io.input.BoundedInputStream + +import org.apache.spark.{SparkEnv, SparkSQLException} +import org.apache.spark.connect.proto +import org.apache.spark.sql.connect.config.Connect +import org.apache.spark.sql.connect.planner.InvalidInputErrors + +object PlanCompressionUtils { + def decompressPlan(plan: proto.Plan): proto.Plan = { + plan.getOpTypeCase match { + case proto.Plan.OpTypeCase.COMPRESSED_OPERATION => + val (cis, closeStream) = decompressBytes( + plan.getCompressedOperation.getData, + plan.getCompressedOperation.getCompressionCodec) + try { + plan.getCompressedOperation.getOpType match { + case proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION => + proto.Plan.newBuilder().setRoot(proto.Relation.parser().parseFrom(cis)).build() + case proto.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND => + proto.Plan.newBuilder().setCommand(proto.Command.parser().parseFrom(cis)).build() + case other => + throw InvalidInputErrors.invalidOneOfField( + other, + plan.getCompressedOperation.getDescriptorForType) + } + } catch { + case e: SparkSQLException => + throw e + case NonFatal(e) => + throw new SparkSQLException( + errorClass = "CONNECT_INVALID_PLAN.CANNOT_PARSE", + messageParameters = Map("errorMsg" -> e.getMessage)) + } finally { + try { + closeStream() + } catch { + case NonFatal(_) => + } + } + case _ => plan + } + } + + private def getMaxPlanSize: Long = { + SparkEnv.get.conf.get(Connect.CONNECT_MAX_PLAN_SIZE) + } + + /** + * Decompress the given bytes using the specified codec. + * @return + * A tuple of decompressed CodedInputStream and a function to close the underlying stream. + */ + private def decompressBytes( + data: ByteString, + compressionCodec: proto.CompressionCodec): (CodedInputStream, () => Unit) = { + compressionCodec match { + case proto.CompressionCodec.COMPRESSION_CODEC_ZSTD => + decompressBytesWithZstd(data, getMaxPlanSize) + case other => + throw InvalidInputErrors.invalidEnum(other) + } + } + + private def decompressBytesWithZstd( + input: ByteString, + maxOutputSize: Long): (CodedInputStream, () => Unit) = { + // Check the declared size in the header against the limit. + val declaredSize = Zstd.getFrameContentSize(input.asReadOnlyByteBuffer()) + if (declaredSize > maxOutputSize) { + throw new SparkSQLException( + errorClass = "CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX", + messageParameters = + Map("planSize" -> declaredSize.toString, "maxPlanSize" -> maxOutputSize.toString)) + } + + val zstdStream = new ZstdInputStreamNoFinalizer(input.newInput()) + + // Create a bounded input stream to limit the decompressed output size to avoid decompression + // bomb attacks. + val boundedStream = new BoundedInputStream(zstdStream, maxOutputSize) { + @throws[IOException] + override protected def onMaxLength(maxBytes: Long, count: Long): Unit = + throw new SparkSQLException( + errorClass = "CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX", + messageParameters = + Map("planSize" -> "unknown", "maxPlanSize" -> maxOutputSize.toString)) + } + val cis = CodedInputStream.newInstance(boundedStream) + cis.setSizeLimit(Integer.MAX_VALUE) + cis.setRecursionLimit(SparkEnv.get.conf.get(Connect.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT)) + (cis, () => boundedStream.close()) + } +} diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala index 6eee71db57093..91dbf419479f3 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala @@ -16,12 +16,16 @@ */ package org.apache.spark.sql.connect.service +import java.io.ByteArrayOutputStream import java.util.UUID +import com.github.luben.zstd.{Zstd, ZstdOutputStreamNoFinalizer} +import com.google.protobuf.ByteString import org.scalatest.concurrent.Eventually import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException +import org.apache.spark.connect.proto import org.apache.spark.sql.connect.SparkConnectServerTest import org.apache.spark.sql.connect.config.Connect @@ -365,4 +369,162 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { assert(error.getMessage.contains("operation_id")) } } + + test("Relation as compressed plan works") { + withClient { client => + val relation = buildPlan("SELECT 1").getRoot + val compressedRelation = Zstd.compress(relation.toByteArray) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val query = client.execute(plan) + while (query.hasNext) query.next() + } + } + + test("Command as compressed plan works") { + withClient { client => + val command = buildSqlCommandPlan("SET spark.sql.session.timeZone=Europe/Berlin").getCommand + val compressedCommand = Zstd.compress(command.toByteArray) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(compressedCommand)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val query = client.execute(plan) + while (query.hasNext) query.next() + } + } + + private def compressInZstdStreamingMode(input: Array[Byte]): Array[Byte] = { + val outputStream = new ByteArrayOutputStream() + val zstdStream = new ZstdOutputStreamNoFinalizer(outputStream) + zstdStream.write(input) + zstdStream.flush() + zstdStream.close() + outputStream.toByteArray + } + + test("Compressed plans generated in streaming mode also work correctly") { + withClient { client => + val relation = buildPlan("SELECT 1").getRoot + val compressedRelation = compressInZstdStreamingMode(relation.toByteArray) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val query = client.execute(plan) + while (query.hasNext) query.next() + } + } + + test("Invalid compressed bytes errors out") { + withClient { client => + val invalidBytes = "invalidBytes".getBytes + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(invalidBytes)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.CANNOT_PARSE")) + } + } + + test("Invalid compressed proto message errors out") { + withClient { client => + val data = Zstd.compress("Apache Spark".getBytes) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(data)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.CANNOT_PARSE")) + } + } + + test("Large compressed plan errors out") { + withClient { client => + withSparkEnvConfs(Connect.CONNECT_MAX_PLAN_SIZE.key -> "100") { + val relation = buildPlan("SELECT '" + "Apache Spark" * 100 + "'").getRoot + val compressedRelation = Zstd.compress(relation.toByteArray) + + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX")) + } + } + } + + test("Large compressed plan generated in streaming mode also errors out") { + withClient { client => + withSparkEnvConfs(Connect.CONNECT_MAX_PLAN_SIZE.key -> "100") { + val relation = buildPlan("SELECT '" + "Apache Spark" * 100 + "'").getRoot + val compressedRelation = compressInZstdStreamingMode(relation.toByteArray) + + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation + .newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX")) + } + } + } } From 6826680393e68fe1cebd1b52998a058a393e47e8 Mon Sep 17 00:00:00 2001 From: Yi Wu Date: Tue, 11 Nov 2025 06:57:19 -0800 Subject: [PATCH 096/400] [SPARK-53898][CORE] Fix race conditions between query cancellation and task completion triggered by eager shuffle cleanup ### What changes were proposed in this pull request? This PR proposes to explicitly handle the `SparkException` thrown by the shuffle statues operations on the non-existent shuffle ID to avoid crashing the `SparkContext`. ### Why are the changes needed? When the main query completes, we cleanup its shuffle statuses and the data files. If there is subquery ongoing before it gets completely cancelled, the subquery can throw `SparkException` from `DAGScheduler` due to the operations (e.g., `MapOutputTrackerMaster.registerMapOutput()`) on the non-existent shuffle ID. And this unexpected exception can crash the `SparkContext`. See the detailed discussion at https://github.com/apache/spark/pull/52213#discussion_r2415632474. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52606 from Ngone51/fix-local-shuffle-cleanup. Lead-authored-by: Yi Wu Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 9a37c3d59b8e80308b626b412d50b114dfe2295d) Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 70 ++++++++----------- .../apache/spark/scheduler/DAGScheduler.scala | 18 +++++ 2 files changed, 49 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 334eb832c4c2b..41a1b51a43154 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -857,37 +857,34 @@ private[spark] class MapOutputTrackerMaster( } } + private def getShuffleStatusOrError(shuffleId: Int, caller: String): ShuffleStatus = { + shuffleStatuses.get(shuffleId) match { + case Some(shuffleStatus) => shuffleStatus + case None => throw new ShuffleStatusNotFoundException(shuffleId, caller) + } + } + def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus): Boolean = { - shuffleStatuses(shuffleId).addMapOutput(mapIndex, status) + getShuffleStatusOrError(shuffleId, "registerMapOutput").addMapOutput(mapIndex, status) } /** Unregister map output information of the given shuffle, mapper and block manager */ def unregisterMapOutput(shuffleId: Int, mapIndex: Int, bmAddress: BlockManagerId): Unit = { - shuffleStatuses.get(shuffleId) match { - case Some(shuffleStatus) => - shuffleStatus.removeMapOutput(mapIndex, bmAddress) - incrementEpoch() - case None => - throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") - } + getShuffleStatusOrError(shuffleId, "unregisterMapOutput").removeMapOutput(mapIndex, bmAddress) + incrementEpoch() } /** Unregister all map and merge output information of the given shuffle. */ def unregisterAllMapAndMergeOutput(shuffleId: Int): Unit = { - shuffleStatuses.get(shuffleId) match { - case Some(shuffleStatus) => - shuffleStatus.removeOutputsByFilter(x => true) - shuffleStatus.removeMergeResultsByFilter(x => true) - shuffleStatus.removeShuffleMergerLocations() - incrementEpoch() - case None => - throw new SparkException( - s"unregisterAllMapAndMergeOutput called for nonexistent shuffle ID $shuffleId.") - } + val shuffleStatus = getShuffleStatusOrError(shuffleId, "unregisterAllMapAndMergeOutput") + shuffleStatus.removeOutputsByFilter(x => true) + shuffleStatus.removeMergeResultsByFilter(x => true) + shuffleStatus.removeShuffleMergerLocations() + incrementEpoch() } def registerMergeResult(shuffleId: Int, reduceId: Int, status: MergeStatus): Unit = { - shuffleStatuses(shuffleId).addMergeResult(reduceId, status) + getShuffleStatusOrError(shuffleId, "registerMergeResult").addMergeResult(reduceId, status) } def registerMergeResults(shuffleId: Int, statuses: Seq[(Int, MergeStatus)]): Unit = { @@ -899,7 +896,8 @@ private[spark] class MapOutputTrackerMaster( def registerShufflePushMergerLocations( shuffleId: Int, shuffleMergers: Seq[BlockManagerId]): Unit = { - shuffleStatuses(shuffleId).registerShuffleMergerLocations(shuffleMergers) + getShuffleStatusOrError(shuffleId, "registerShufflePushMergerLocations") + .registerShuffleMergerLocations(shuffleMergers) } /** @@ -918,28 +916,19 @@ private[spark] class MapOutputTrackerMaster( reduceId: Int, bmAddress: BlockManagerId, mapIndex: Option[Int] = None): Unit = { - shuffleStatuses.get(shuffleId) match { - case Some(shuffleStatus) => - val mergeStatus = shuffleStatus.mergeStatuses(reduceId) - if (mergeStatus != null && - (mapIndex.isEmpty || mergeStatus.tracker.contains(mapIndex.get))) { - shuffleStatus.removeMergeResult(reduceId, bmAddress) - incrementEpoch() - } - case None => - throw new SparkException("unregisterMergeResult called for nonexistent shuffle ID") + val shuffleStatus = getShuffleStatusOrError(shuffleId, "unregisterMergeResult") + val mergeStatus = shuffleStatus.mergeStatuses(reduceId) + if (mergeStatus != null && + (mapIndex.isEmpty || mergeStatus.tracker.contains(mapIndex.get))) { + shuffleStatus.removeMergeResult(reduceId, bmAddress) + incrementEpoch() } } def unregisterAllMergeResult(shuffleId: Int): Unit = { - shuffleStatuses.get(shuffleId) match { - case Some(shuffleStatus) => - shuffleStatus.removeMergeResultsByFilter(x => true) - incrementEpoch() - case None => - throw new SparkException( - s"unregisterAllMergeResult called for nonexistent shuffle ID $shuffleId.") - } + getShuffleStatusOrError(shuffleId, "unregisterAllMergeResult") + .removeMergeResultsByFilter(x => true) + incrementEpoch() } /** Unregister shuffle data */ @@ -1022,7 +1011,7 @@ private[spark] class MapOutputTrackerMaster( * Return statistics about all of the outputs for a given shuffle. */ def getStatistics(dep: ShuffleDependency[_, _, _]): MapOutputStatistics = { - shuffleStatuses(dep.shuffleId).withMapStatuses { statuses => + getShuffleStatusOrError(dep.shuffleId, "getStatistics").withMapStatuses { statuses => val totalSizes = new Array[Long](dep.partitioner.numPartitions) val parallelAggThreshold = conf.get( SHUFFLE_MAP_OUTPUT_PARALLEL_AGGREGATION_THRESHOLD) @@ -1285,6 +1274,9 @@ private[spark] class MapOutputTrackerMaster( } } +case class ShuffleStatusNotFoundException(shuffleId: Int, methodName: String) + extends SparkException(s"$methodName called for nonexistent shuffle ID $shuffleId.") + /** * Executor-side client for fetching map output info from the driver's MapOutputTrackerMaster. * Note that this is not used in local-mode; instead, local-mode Executors access the diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7d77628c3f088..7c8bea31334b1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -2920,6 +2920,21 @@ private[spark] class DAGScheduler( } } + private[scheduler] def handleShuffleStatusNotFoundException( + ex: ShuffleStatusNotFoundException): Unit = { + val stage = shuffleIdToMapStage.get(ex.shuffleId) + val reason = "exceptions encountered while invoking " + + s"MapOutputTracker.${ex.methodName} with shuffleId=${ex.shuffleId}" + if (stage.isDefined) { + abortStage(stage.get, reason, Some(ex)) + logWarning(s"Aborting stage because of $reason. It is possible that the stage is " + + "being cancelled.") + } else { + logWarning(s"Tried aborting stage because of $reason, but the stage was not found. " + + "It is possible that the stage has been cancelled earlier.") + } + } + /** * Marks a stage as finished and removes it from the list of running stages. */ @@ -3192,6 +3207,9 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler val timerContext = timer.time() try { doOnReceive(event) + } catch { + case ex: ShuffleStatusNotFoundException => + dagScheduler.handleShuffleStatusNotFoundException(ex) } finally { timerContext.stop() } From adfdf32b23fe9197af36e0e4c126fbce39d7f241 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 4 Nov 2025 12:22:29 -0800 Subject: [PATCH 097/400] [SPARK-53991][SQL] Add SQL support for KLL quantiles functions based on DataSketches ### What changes were proposed in this pull request? This PR adds support for KLL (K-Linear-Logarithmic) quantile sketches to Spark SQL, based on the Apache DataSketches KLL library. KLL sketches provide a compact, approximate representation of data distributions, enabling efficient quantile estimation and rank queries on large datasets with bounded memory usage and strong accuracy guarantees. Jira: https://issues.apache.org/jira/browse/SPARK-53991. It introduces 18 new SQL functions organized into six categories: 1. Aggregation Functions Creates a KLL sketch from input values. ``` kll_sketch_agg_bigint(col) kll_sketch_agg_float(col) kll_sketch_agg_double(col) ``` 2. Sketch Inspection Functions Returns a human-readable string representation for debugging purposes. ``` kll_sketch_to_string_bigint(sketch) kll_sketch_to_string_float(sketch) kll_sketch_to_string_double(sketch) ``` 3. Sketch Merging Functions Merges two compatible sketches. ``` kll_sketch_merge_bigint(sketch1, sketch2) kll_sketch_merge_float(sketch1, sketch2) kll_sketch_merge_double(sketch1, sketch2) ``` 4. Quantile Estimation Functions Estimates the value at a given rank, supporting both single rank values and arrays of ranks for batch quantile queries. ``` kll_sketch_get_quantile_bigint(sketch, rank) kll_sketch_get_quantile_float(sketch, rank) kll_sketch_get_quantile_double(sketch, rank) ``` 5. Rank Estimation Functions Estimates the rank of a given value, supporting both single values and arrays of values for batch rank queries. ``` kll_sketch_get_rank_bigint(sketch, value) kll_sketch_get_rank_float(sketch, value) kll_sketch_get_rank_double(sketch, value) ``` 6. Sketch Item Count Functions Counts the number of items collected in the sketch so far. ``` kll_sketch_get_n_bigint(sketch) kll_sketch_get_n_float(sketch) kll_sketch_get_n_double(sketch) ``` This PR only includes SQL language support; Dataframe API support will follow in a separate PR. Key Features: * Type Safety: Separate implementations for BIGINT (covering TINYINT/SMALLINT/INT), FLOAT, and DOUBLE types ensure type-safe operations * Array Support: Quantile and rank functions accept arrays for efficient batch operations * Memory Efficient: Sketches are serialized to BINARY type for compact storage and efficient shuffling * NULL Handling: All aggregate functions properly ignore NULL input values, consistent with standard SQL aggregate behavior * Error Handling: Comprehensive validation with structured error messages for: invalid quantile ranges (must be 0.0-1.0), incompatible sketch merges, invalid binary sketch data, type mismatches ### Why are the changes needed? KLL sketches enable approximate quantile and rank queries on large datasets with: * O(1) space complexity - Bounded memory usage regardless of data size * High accuracy - Configurable error bounds with proven theoretical guarantees * Fast queries - O(log n) query time for quantile/rank estimation * Mergeable - Sketches can be combined for distributed aggregation Use cases include: * Approximate median/percentile calculations on massive datasets * Distribution analysis for monitoring and analytics * SLA compliance checking (e.g., p95, p99 latency) * Efficient histogram generation ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces 15 new SQL functions available in Spark SQL. ### How was this patch tested? SQL Golden File Tests: Added `kllquantiles.sql` with test queries covering: * All three data types (BIGINT, FLOAT, DOUBLE) * Multiple input sizes (empty, single value, multiple values) * NULL value handling (verified NULLs are ignored) * Quantile estimation (single and array inputs) * Rank estimation (single and array inputs) * Sketch merging * Approximate result validation using tolerance-based comparisons * Negative tests for error conditions (invalid quantiles, type mismatches, incompatible merges) ### Was this patch authored or co-authored using generative AI tooling? Yes, code assistance with `claude-4.5-sonnet` in combination with manual editing by the author. Closes #52800 from dtenedor/kll-quantiles-functions. Authored-by: Daniel Tenedorio Signed-off-by: Daniel Tenedorio --- .../resources/error/error-conditions.json | 30 + .../catalyst/analysis/FunctionRegistry.scala | 18 + .../expressions/aggregate/kllAggregates.scala | 512 +++++++ .../catalyst/expressions/kllExpressions.scala | 671 ++++++++++ .../sql/errors/QueryExecutionErrors.scala | 38 + .../sql-functions/sql-expression-schema.md | 18 + .../analyzer-results/kllquantiles.sql.out | 1124 ++++++++++++++++ .../sql-tests/inputs/kllquantiles.sql | 422 ++++++ .../sql-tests/results/kllquantiles.sql.out | 1187 +++++++++++++++++ 9 files changed, 4020 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 4f1292efc1a05..4b07c2624d5d5 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4188,6 +4188,36 @@ ], "sqlState" : "42K0E" }, + "KLL_SKETCH_INCOMPATIBLE_MERGE" : { + "message" : [ + "For function , cannot merge KLL sketches: " + ], + "sqlState" : "22000" + }, + "KLL_SKETCH_INVALID_INPUT" : { + "message" : [ + "For function , invalid KLL sketch binary data: " + ], + "sqlState" : "22000" + }, + "KLL_SKETCH_INVALID_QUANTILE_RANGE" : { + "message" : [ + "For function , the quantile value must be between 0.0 and 1.0 (inclusive), but got ." + ], + "sqlState" : "22003" + }, + "KLL_SKETCH_K_MUST_BE_CONSTANT" : { + "message" : [ + "For function , the k parameter must be a constant value, but got a non-constant expression." + ], + "sqlState" : "42K0E" + }, + "KLL_SKETCH_K_OUT_OF_RANGE" : { + "message" : [ + "For function , the k parameter must be between 8 and 65535 (inclusive), but got ." + ], + "sqlState" : "22003" + }, "KRYO_BUFFER_OVERFLOW" : { "message" : [ "Kryo serialization failed: . To avoid this, increase \"\" value." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 8ded5873f14a2..e6e713fab7d9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -536,6 +536,24 @@ object FunctionRegistry { expression[ThetaIntersectionAgg]("theta_intersection_agg"), expression[ApproxTopKAccumulate]("approx_top_k_accumulate"), expression[ApproxTopKCombine]("approx_top_k_combine"), + expression[KllSketchAggBigint]("kll_sketch_agg_bigint"), + expression[KllSketchAggFloat]("kll_sketch_agg_float"), + expression[KllSketchAggDouble]("kll_sketch_agg_double"), + expression[KllSketchToStringBigint]("kll_sketch_to_string_bigint"), + expression[KllSketchToStringFloat]("kll_sketch_to_string_float"), + expression[KllSketchToStringDouble]("kll_sketch_to_string_double"), + expression[KllSketchGetNBigint]("kll_sketch_get_n_bigint"), + expression[KllSketchGetNFloat]("kll_sketch_get_n_float"), + expression[KllSketchGetNDouble]("kll_sketch_get_n_double"), + expression[KllSketchMergeBigint]("kll_sketch_merge_bigint"), + expression[KllSketchMergeFloat]("kll_sketch_merge_float"), + expression[KllSketchMergeDouble]("kll_sketch_merge_double"), + expression[KllSketchGetQuantileBigint]("kll_sketch_get_quantile_bigint"), + expression[KllSketchGetQuantileFloat]("kll_sketch_get_quantile_float"), + expression[KllSketchGetQuantileDouble]("kll_sketch_get_quantile_double"), + expression[KllSketchGetRankBigint]("kll_sketch_get_rank_bigint"), + expression[KllSketchGetRankFloat]("kll_sketch_get_rank_float"), + expression[KllSketchGetRankDouble]("kll_sketch_get_rank_double"), // string functions expression[Ascii]("ascii"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala new file mode 100644 index 0000000000000..5891155cf7537 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala @@ -0,0 +1,512 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.datasketches.kll.{KllDoublesSketch, KllFloatsSketch, KllLongsSketch} +import org.apache.datasketches.memory.Memory + +import org.apache.spark.SparkUnsupportedOperationException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types.{AbstractDataType, BinaryType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ShortType, TypeCollection} + +/** + * The KllSketchAggBigint function utilizes an Apache DataSketches KllLongsSketch instance to + * compute quantiles of the values of an input expression (such as an input column in a table). + * It outputs the binary representation of the KllLongsSketch. + * + * See [[https://datasketches.apache.org/docs/KLL/KLLSketch.html]] for more information. + * + * @param child + * child expression against which quantile computation will occur + * @param kExpr + * optional expression for the k parameter from the Apache DataSketches library that controls + * the size and accuracy of the sketch. Must be a constant integer between 8 and 65535. + * Default is 200 (normalized rank error ~1.65%). Larger k values provide more accurate + * estimates but result in larger, slower sketches. + * @param mutableAggBufferOffset + * offset for mutable aggregation buffer + * @param inputAggBufferOffset + * offset for input aggregation buffer + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, k]) - Returns the KllLongsSketch compact binary representation. + The optional k parameter controls the size and accuracy of the sketch (default 200, range 8-65535). + Larger k values provide more accurate quantile estimates but result in larger, slower sketches. + """, + examples = """ + Examples: + > SELECT LENGTH(kll_sketch_to_string_bigint(_FUNC_(col))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col); + true + > SELECT LENGTH(kll_sketch_to_string_bigint(_FUNC_(col, 400))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col); + true + """, + group = "agg_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class KllSketchAggBigint( + child: Expression, + kExpr: Option[Expression] = None, + override val mutableAggBufferOffset: Int = 0, + override val inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[KllLongsSketch] + with KllSketchAggBase + with ExpectsInputTypes { + def this(child: Expression) = this(child, None, 0, 0) + def this(child: Expression, kExpr: Expression) = this(child, Some(kExpr), 0, 0) + + override def children: Seq[Expression] = child +: kExpr.toSeq + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): KllSketchAggBigint = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + override def withNewInputAggBufferOffset( + newInputAggBufferOffset: Int): KllSketchAggBigint = + copy(inputAggBufferOffset = newInputAggBufferOffset) + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): KllSketchAggBigint = { + if (newChildren.length == 1) { + copy(child = newChildren(0), kExpr = None) + } else { + copy(child = newChildren(0), kExpr = Some(newChildren(1))) + } + } + + override def dataType: DataType = BinaryType + override def inputTypes: Seq[AbstractDataType] = { + val baseTypes = Seq( + TypeCollection( + ByteType, + IntegerType, + LongType, + ShortType)) + if (kExpr.isDefined) baseTypes :+ IntegerType else baseTypes + } + override def nullable: Boolean = false + override def prettyName: String = "kll_sketch_agg_bigint" + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + return defaultCheck + } + checkKInputDataTypes() + } + + override def createAggregationBuffer(): KllLongsSketch = + KllLongsSketch.newHeapInstance(kValue) + + /** + * Evaluate the input row and update the KllLongsSketch instance with the row's value. The update + * function only supports a subset of Spark SQL types, and an exception will be thrown for + * unsupported types. + * Note, null values are ignored. + */ + override def update(sketch: KllLongsSketch, input: InternalRow): KllLongsSketch = { + // Return early for null values. + val v = child.eval(input) + if (v == null) { + return sketch + } + // Handle the different data types for sketch updates. + child.dataType match { + case ByteType => + sketch.update(v.asInstanceOf[Byte].toLong) + case IntegerType => + sketch.update(v.asInstanceOf[Int].toLong) + case LongType => + sketch.update(v.asInstanceOf[Long]) + case ShortType => + sketch.update(v.asInstanceOf[Short].toLong) + case _ => + throw unexpectedInputDataTypeError(child) + } + + sketch + } + + /** Merges an input sketch into the current aggregation buffer. */ + override def merge(updateBuffer: KllLongsSketch, input: KllLongsSketch): KllLongsSketch = { + try { + updateBuffer.merge(input) + updateBuffer + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + } + } + + /** Returns a sketch derived from the input column or expression. */ + override def eval(sketch: KllLongsSketch): Any = sketch.toByteArray + + /** Converts the underlying sketch state into a byte array. */ + override def serialize(sketch: KllLongsSketch): Array[Byte] = sketch.toByteArray + + /** Wraps the byte array into a sketch instance. */ + override def deserialize(buffer: Array[Byte]): KllLongsSketch = if (buffer.nonEmpty) { + try { + KllLongsSketch.heapify(Memory.wrap(buffer)) + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } else { + this.createAggregationBuffer() + } +} + +/** + * The KllSketchAggFloat function utilizes an Apache DataSketches KllFloatsSketch instance to + * compute quantiles of the values of an input expression (such as an input column in a table). + * It outputs the binary representation of the KllFloatsSketch. + * + * See [[https://datasketches.apache.org/docs/KLL/KLLSketch.html]] for more information. + * + * @param child + * child expression against which quantile computation will occur + * @param kExpr + * optional expression for the k parameter from the Apache DataSketches library that controls + * the size and accuracy of the sketch. Must be a constant integer between 8 and 65535. + * Default is 200 (normalized rank error ~1.65%). Larger k values provide more accurate + * estimates but result in larger, slower sketches. + * @param mutableAggBufferOffset + * offset for mutable aggregation buffer + * @param inputAggBufferOffset + * offset for input aggregation buffer + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, k]) - Returns the KllFloatsSketch compact binary representation. + The optional k parameter controls the size and accuracy of the sketch (default 200, range 8-65535). + Larger k values provide more accurate quantile estimates but result in larger, slower sketches. + """, + examples = """ + Examples: + > SELECT LENGTH(kll_sketch_to_string_float(_FUNC_(col))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + true + > SELECT LENGTH(kll_sketch_to_string_float(_FUNC_(col, 400))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + true + """, + group = "agg_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class KllSketchAggFloat( + child: Expression, + kExpr: Option[Expression] = None, + override val mutableAggBufferOffset: Int = 0, + override val inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[KllFloatsSketch] + with KllSketchAggBase + with ExpectsInputTypes { + def this(child: Expression) = this(child, None, 0, 0) + def this(child: Expression, kExpr: Expression) = this(child, Some(kExpr), 0, 0) + + override def children: Seq[Expression] = child +: kExpr.toSeq + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): KllSketchAggFloat = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + override def withNewInputAggBufferOffset( + newInputAggBufferOffset: Int): KllSketchAggFloat = + copy(inputAggBufferOffset = newInputAggBufferOffset) + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): KllSketchAggFloat = { + if (newChildren.length == 1) { + copy(child = newChildren(0), kExpr = None) + } else { + copy(child = newChildren(0), kExpr = Some(newChildren(1))) + } + } + + override def dataType: DataType = BinaryType + override def inputTypes: Seq[AbstractDataType] = { + val baseTypes = Seq(FloatType) + if (kExpr.isDefined) baseTypes :+ IntegerType else baseTypes + } + override def nullable: Boolean = false + override def prettyName: String = "kll_sketch_agg_float" + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + return defaultCheck + } + checkKInputDataTypes() + } + + override def createAggregationBuffer(): KllFloatsSketch = + KllFloatsSketch.newHeapInstance(kValue) + + /** + * Evaluate the input row and update the KllFloatsSketch instance with the row's value. The update + * function only supports FloatType to avoid precision loss from integer-to-float conversion. + * Users should use kll_sketch_agg_bigint for integer types. + * Note, Null values are ignored. + */ + override def update(sketch: KllFloatsSketch, input: InternalRow): KllFloatsSketch = { + // Return early for null values. + val v = child.eval(input) + if (v == null) { + return sketch + } + // Handle the different data types for sketch updates. + child.dataType match { + case FloatType => + sketch.update(v.asInstanceOf[Float]) + case _ => + throw unexpectedInputDataTypeError(child) + } + + sketch + } + + /** Merges an input sketch into the current aggregation buffer. */ + override def merge(updateBuffer: KllFloatsSketch, input: KllFloatsSketch): KllFloatsSketch = { + try { + updateBuffer.merge(input) + updateBuffer + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + } + } + + /** Returns a sketch derived from the input column or expression. */ + override def eval(sketch: KllFloatsSketch): Any = sketch.toByteArray + + /** Converts the underlying sketch state into a byte array. */ + override def serialize(sketch: KllFloatsSketch): Array[Byte] = sketch.toByteArray + + /** Wraps the byte array into a sketch instance. */ + override def deserialize(buffer: Array[Byte]): KllFloatsSketch = if (buffer.nonEmpty) { + try { + KllFloatsSketch.heapify(Memory.wrap(buffer)) + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } else { + this.createAggregationBuffer() + } +} + +/** + * The KllSketchAggDouble function utilizes an Apache DataSketches KllDoublesSketch instance to + * compute quantiles of the values of an input expression (such as an input column in a table). + * It outputs the binary representation of the KllDoublesSketch. + * + * See [[https://datasketches.apache.org/docs/KLL/KLLSketch.html]] for more information. + * + * @param child + * child expression against which quantile computation will occur + * @param kExpr + * optional expression for the k parameter from the Apache DataSketches library that controls + * the size and accuracy of the sketch. Must be a constant integer between 8 and 65535. + * Default is 200 (normalized rank error ~1.65%). Larger k values provide more accurate + * estimates but result in larger, slower sketches. + * @param mutableAggBufferOffset + * offset for mutable aggregation buffer + * @param inputAggBufferOffset + * offset for input aggregation buffer + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, k]) - Returns the KllDoublesSketch compact binary representation. + The optional k parameter controls the size and accuracy of the sketch (default 200, range 8-65535). + Larger k values provide more accurate quantile estimates but result in larger, slower sketches. + """, + examples = """ + Examples: + > SELECT LENGTH(kll_sketch_to_string_double(_FUNC_(col))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + true + > SELECT LENGTH(kll_sketch_to_string_double(_FUNC_(col, 400))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + true + """, + group = "agg_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class KllSketchAggDouble( + child: Expression, + kExpr: Option[Expression] = None, + override val mutableAggBufferOffset: Int = 0, + override val inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[KllDoublesSketch] + with KllSketchAggBase + with ExpectsInputTypes { + def this(child: Expression) = this(child, None, 0, 0) + def this(child: Expression, kExpr: Expression) = this(child, Some(kExpr), 0, 0) + + override def children: Seq[Expression] = child +: kExpr.toSeq + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): KllSketchAggDouble = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + override def withNewInputAggBufferOffset( + newInputAggBufferOffset: Int): KllSketchAggDouble = + copy(inputAggBufferOffset = newInputAggBufferOffset) + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): KllSketchAggDouble = { + if (newChildren.length == 1) { + copy(child = newChildren(0), kExpr = None) + } else { + copy(child = newChildren(0), kExpr = Some(newChildren(1))) + } + } + + override def dataType: DataType = BinaryType + override def inputTypes: Seq[AbstractDataType] = { + val baseTypes = Seq(TypeCollection(FloatType, DoubleType)) + if (kExpr.isDefined) baseTypes :+ IntegerType else baseTypes + } + override def nullable: Boolean = false + override def prettyName: String = "kll_sketch_agg_double" + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + return defaultCheck + } + checkKInputDataTypes() + } + + override def createAggregationBuffer(): KllDoublesSketch = + KllDoublesSketch.newHeapInstance(kValue) + + /** + * Evaluate the input row and update the KllDoublesSketch instance with the row's value. + * The update function only supports FloatType and DoubleType to avoid precision loss from + * integer-to-double conversion. Users should use kll_sketch_agg_bigint for integer types. + * Note, Null values are ignored. + */ + override def update(sketch: KllDoublesSketch, input: InternalRow): KllDoublesSketch = { + // Return early for null values. + val v = child.eval(input) + if (v == null) { + return sketch + } + // Handle the different data types for sketch updates. + child.dataType match { + case DoubleType => + sketch.update(v.asInstanceOf[Double]) + case FloatType => + sketch.update(v.asInstanceOf[Float].toDouble) + case _ => + throw unexpectedInputDataTypeError(child) + } + + sketch + } + + /** Merges an input sketch into the current aggregation buffer. */ + override def merge(updateBuffer: KllDoublesSketch, input: KllDoublesSketch): KllDoublesSketch = { + try { + updateBuffer.merge(input) + updateBuffer + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + } + } + + /** Returns a sketch derived from the input column or expression. */ + override def eval(sketch: KllDoublesSketch): Any = sketch.toByteArray + + /** Converts the underlying sketch state into a byte array. */ + override def serialize(sketch: KllDoublesSketch): Array[Byte] = sketch.toByteArray + + /** Wraps the byte array into a sketch instance. */ + override def deserialize(buffer: Array[Byte]): KllDoublesSketch = if (buffer.nonEmpty) { + try { + KllDoublesSketch.heapify(Memory.wrap(buffer)) + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } else { + this.createAggregationBuffer() + } +} + +/** + * Common trait for KLL sketch aggregate functions that support an optional k parameter. + */ +trait KllSketchAggBase { + def kExpr: Option[Expression] + def prettyName: String + + // Constants from the Apache DataSketches library. + private val MIN_K = 8 + private val MAX_K = 65535 + private val DEFAULT_K = 200 + + // Validate and extract k value + protected lazy val kValue: Int = { + kExpr match { + case Some(expr) => + if (!expr.foldable) { + throw QueryExecutionErrors.kllSketchKMustBeConstantError(prettyName) + } + val k = expr.eval().asInstanceOf[Int] + if (k < MIN_K || k > MAX_K) { + throw QueryExecutionErrors.kllSketchKOutOfRangeError(prettyName, k) + } + k + case None => DEFAULT_K + } + } + + protected def checkKInputDataTypes(): TypeCheckResult = { + kExpr match { + case Some(expr) => + if (!expr.foldable) { + DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> "k", + "inputType" -> "int", + "inputExpr" -> expr.sql)) + } else if (expr.eval() == null) { + DataTypeMismatch( + errorSubClass = "UNEXPECTED_NULL", + messageParameters = Map("exprName" -> "k")) + } else { + // Trigger validation + try { + kValue + TypeCheckResult.TypeCheckSuccess + } catch { + case e: Exception => TypeCheckResult.TypeCheckFailure(e.getMessage) + } + } + case None => TypeCheckResult.TypeCheckSuccess + } + } + + protected def unexpectedInputDataTypeError( + child: Expression): SparkUnsupportedOperationException = + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_3121", + messageParameters = Map("dataType" -> child.dataType.toString)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala new file mode 100644 index 0000000000000..18a9fc6e1f19f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala @@ -0,0 +1,671 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.datasketches.kll.{KllDoublesSketch, KllFloatsSketch, KllLongsSketch} +import org.apache.datasketches.memory.Memory + +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types.{AbstractDataType, ArrayType, BinaryType, DataType, DoubleType, FloatType, LongType, StringType, TypeCollection} +import org.apache.spark.unsafe.types.UTF8String + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns human readable summary information about this sketch. + """, + examples = """ + Examples: + > SELECT LENGTH(_FUNC_(kll_sketch_agg_bigint(col))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchToStringBigint(child: Expression) extends KllSketchToStringBase { + override protected def withNewChildInternal(newChild: Expression): KllSketchToStringBigint = + copy(child = newChild) + override def prettyName: String = "kll_sketch_to_string_bigint" + override def nullSafeEval(input: Any): Any = { + try { + val buffer = input.asInstanceOf[Array[Byte]] + val sketch = KllLongsSketch.heapify(Memory.wrap(buffer)) + UTF8String.fromString(sketch.toString()) + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns human readable summary information about this sketch. + """, + examples = """ + Examples: + > SELECT LENGTH(_FUNC_(kll_sketch_agg_float(col))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchToStringFloat(child: Expression) extends KllSketchToStringBase { + override protected def withNewChildInternal(newChild: Expression): KllSketchToStringFloat = + copy(child = newChild) + override def prettyName: String = "kll_sketch_to_string_float" + override def nullSafeEval(input: Any): Any = { + try { + val buffer = input.asInstanceOf[Array[Byte]] + val sketch = KllFloatsSketch.heapify(Memory.wrap(buffer)) + UTF8String.fromString(sketch.toString()) + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns human readable summary information about this sketch. + """, + examples = """ + Examples: + > SELECT LENGTH(_FUNC_(kll_sketch_agg_double(col))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchToStringDouble(child: Expression) extends KllSketchToStringBase { + override protected def withNewChildInternal(newChild: Expression): KllSketchToStringDouble = + copy(child = newChild) + override def prettyName: String = "kll_sketch_to_string_double" + override def nullSafeEval(input: Any): Any = { + try { + val buffer = input.asInstanceOf[Array[Byte]] + val sketch = KllDoublesSketch.heapify(Memory.wrap(buffer)) + UTF8String.fromString(sketch.toString()) + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } +} + +/** This is a base class for the above expressions to reduce boilerplate. */ +abstract class KllSketchToStringBase + extends UnaryExpression + with CodegenFallback + with ImplicitCastInputTypes { + override def dataType: DataType = StringType + override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) + override def nullIntolerant: Boolean = true +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns the number of items collected in the sketch. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_bigint(col)) FROM VALUES (1), (2), (3), (4), (5) tab(col); + 5 + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetNBigint(child: Expression) extends KllSketchGetNBase { + override protected def withNewChildInternal(newChild: Expression): KllSketchGetNBigint = + copy(child = newChild) + override def prettyName: String = "kll_sketch_get_n_bigint" + override def nullSafeEval(input: Any): Any = { + try { + val buffer = input.asInstanceOf[Array[Byte]] + val sketch = KllLongsSketch.heapify(Memory.wrap(buffer)) + sketch.getN() + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns the number of items collected in the sketch. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_float(col)) FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + 5 + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetNFloat(child: Expression) extends KllSketchGetNBase { + override protected def withNewChildInternal(newChild: Expression): KllSketchGetNFloat = + copy(child = newChild) + override def prettyName: String = "kll_sketch_get_n_float" + override def nullSafeEval(input: Any): Any = { + try { + val buffer = input.asInstanceOf[Array[Byte]] + val sketch = KllFloatsSketch.heapify(Memory.wrap(buffer)) + sketch.getN() + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns the number of items collected in the sketch. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_double(col)) FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + 5 + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetNDouble(child: Expression) extends KllSketchGetNBase { + override protected def withNewChildInternal(newChild: Expression): KllSketchGetNDouble = + copy(child = newChild) + override def prettyName: String = "kll_sketch_get_n_double" + override def nullSafeEval(input: Any): Any = { + try { + val buffer = input.asInstanceOf[Array[Byte]] + val sketch = KllDoublesSketch.heapify(Memory.wrap(buffer)) + sketch.getN() + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } +} + +/** This is a base class for the above expressions to reduce boilerplate. */ +abstract class KllSketchGetNBase + extends UnaryExpression + with CodegenFallback + with ImplicitCastInputTypes { + override def dataType: DataType = LongType + override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) + override def nullIntolerant: Boolean = true +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Merges two sketch buffers together into one. + """, + examples = """ + Examples: + > SELECT LENGTH(kll_sketch_to_string_bigint(_FUNC_(kll_sketch_agg_bigint(col), kll_sketch_agg_bigint(col)))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchMergeBigint(left: Expression, right: Expression) extends KllSketchMergeBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_merge_bigint" + override def nullSafeEval(left: Any, right: Any): Any = { + try { + val leftBuffer = left.asInstanceOf[Array[Byte]] + val rightBuffer = right.asInstanceOf[Array[Byte]] + val leftSketch = KllLongsSketch.heapify(Memory.wrap(leftBuffer)) + val rightSketch = KllLongsSketch.wrap(Memory.wrap(rightBuffer)) + leftSketch.merge(rightSketch) + leftSketch.toByteArray + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Merges two sketch buffers together into one. + """, + examples = """ + Examples: + > SELECT LENGTH(kll_sketch_to_string_float(_FUNC_(kll_sketch_agg_float(col), kll_sketch_agg_float(col)))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchMergeFloat(left: Expression, right: Expression) extends KllSketchMergeBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_merge_float" + override def nullSafeEval(left: Any, right: Any): Any = { + try { + val leftBuffer = left.asInstanceOf[Array[Byte]] + val rightBuffer = right.asInstanceOf[Array[Byte]] + val leftSketch = KllFloatsSketch.heapify(Memory.wrap(leftBuffer)) + val rightSketch = KllFloatsSketch.wrap(Memory.wrap(rightBuffer)) + leftSketch.merge(rightSketch) + leftSketch.toByteArray + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Merges two sketch buffers together into one. + """, + examples = """ + Examples: + > SELECT LENGTH(kll_sketch_to_string_double(_FUNC_(kll_sketch_agg_double(col), kll_sketch_agg_double(col)))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchMergeDouble(left: Expression, right: Expression) extends KllSketchMergeBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_merge_double" + override def nullSafeEval(left: Any, right: Any): Any = { + try { + val leftBuffer = left.asInstanceOf[Array[Byte]] + val rightBuffer = right.asInstanceOf[Array[Byte]] + val leftSketch = KllDoublesSketch.heapify(Memory.wrap(leftBuffer)) + val rightSketch = KllDoublesSketch.wrap(Memory.wrap(rightBuffer)) + leftSketch.merge(rightSketch) + leftSketch.toByteArray + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + } + } +} + +/** This is a base class for the above expressions to reduce boilerplate. */ +abstract class KllSketchMergeBase + extends BinaryExpression + with CodegenFallback + with ImplicitCastInputTypes { + override def dataType: DataType = BinaryType + override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, BinaryType) + override def nullIntolerant: Boolean = true +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Extracts a single value from the quantiles sketch representing the + desired quantile given the input rank. The desired quantile can either be a single value + or an array. In the latter case, the function will return an array of results of equal + length to the input array. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_bigint(col), 0.5) > 1 FROM VALUES (1), (2), (3), (4), (5) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetQuantileBigint(left: Expression, right: Expression) + extends KllSketchGetQuantileBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_get_quantile_bigint" + override def outputDataType: DataType = LongType + override def kllSketchGetQuantile(memory: Memory, rank: Double): Any = { + withQuantileErrorHandling(rank) { + KllLongsSketch.wrap(memory).getQuantile(rank) + } + } + override def kllSketchGetQuantiles(memory: Memory, ranks: Array[Double]): Array[Any] = { + withQuantileErrorHandling(if (ranks.length > 0) ranks(0) else 0.0) { + KllLongsSketch.wrap(memory).getQuantiles(ranks).map(_.asInstanceOf[Any]) + } + } +} + +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Extracts a single value from the quantiles sketch representing the + desired quantile given the input rank. The desired quantile can either be a single value + or an array. In the latter case, the function will return an array of results of equal + length to the input array. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_float(col), 0.5) > 1 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetQuantileFloat(left: Expression, right: Expression) + extends KllSketchGetQuantileBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_get_quantile_float" + override def outputDataType: DataType = FloatType + override def kllSketchGetQuantile(memory: Memory, rank: Double): Any = { + withQuantileErrorHandling(rank) { + KllFloatsSketch.wrap(memory).getQuantile(rank) + } + } + override def kllSketchGetQuantiles(memory: Memory, ranks: Array[Double]): Array[Any] = { + withQuantileErrorHandling(if (ranks.length > 0) ranks(0) else 0.0) { + KllFloatsSketch.wrap(memory).getQuantiles(ranks).map(_.asInstanceOf[Any]) + } + } +} + +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Extracts a single value from the quantiles sketch representing the + desired quantile given the input rank. The desired quantile can either be a single value + or an array. In the latter case, the function will return an array of results of equal + length to the input array. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_double(col), 0.5) > 1 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetQuantileDouble(left: Expression, right: Expression) + extends KllSketchGetQuantileBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_get_quantile_double" + override def outputDataType: DataType = DoubleType + override def kllSketchGetQuantile(memory: Memory, rank: Double): Any = { + withQuantileErrorHandling(rank) { + KllDoublesSketch.wrap(memory).getQuantile(rank) + } + } + override def kllSketchGetQuantiles(memory: Memory, ranks: Array[Double]): Array[Any] = { + withQuantileErrorHandling(if (ranks.length > 0) ranks(0) else 0.0) { + KllDoublesSketch.wrap(memory).getQuantiles(ranks).map(_.asInstanceOf[Any]) + } + } +} + +/** + * This is a base class for the above expressions to reduce boilerplate. + * Each implementor is expected to define three methods: one to specify the output data type, + * one to compute the quantile of an input sketch buffer given a single input rank, + * and one to compute multiple quantiles given an array of ranks (batch API for performance). + */ +abstract class KllSketchGetQuantileBase + extends BinaryExpression + with CodegenFallback + with ImplicitCastInputTypes { + /** + * This method accepts a KLL quantiles Memory segment, wraps it with the corresponding + * Kll*Sketch.wrap method, and then calls getQuantile on the result. + * @param memory The input KLL quantiles sketch buffer to extract the quantile from + * @param rank The input rank to use to compute the quantile + * @return The result quantile + */ + protected def kllSketchGetQuantile(memory: Memory, rank: Double): Any + + /** + * This method accepts a KLL quantiles Memory segment, wraps it with the corresponding + * Kll*Sketch.wrap method, and then calls getQuantiles on the result (batch API). + * @param memory The input KLL quantiles sketch buffer to extract the quantiles from + * @param ranks The input ranks array to use to compute the quantiles + * @return The result quantiles as an array + */ + protected def kllSketchGetQuantiles(memory: Memory, ranks: Array[Double]): Array[Any] + + /** + * Helper method to wrap quantile operations with consistent error handling. + * @param rankForError The rank value to include in error messages + * @param operation The operation to execute + * @return The result of the operation + */ + protected def withQuantileErrorHandling[T](rankForError: Double)(operation: => T): T = { + try { + operation + } catch { + case e: org.apache.datasketches.common.SketchesArgumentException => + if (e.getMessage.contains("normalized rank")) { + throw QueryExecutionErrors.kllSketchInvalidQuantileRangeError(prettyName, rankForError) + } else { + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } + + /** The output data type for a single value (not array) */ + protected def outputDataType: DataType + + override def nullIntolerant: Boolean = true + override def inputTypes: Seq[AbstractDataType] = + Seq( + BinaryType, + TypeCollection( + DoubleType, + ArrayType(DoubleType, containsNull = false))) + + override def dataType: DataType = { + right.dataType match { + case ArrayType(_, _) => ArrayType(outputDataType, false) + case _ => outputDataType + } + } + + override def nullSafeEval(leftInput: Any, rightInput: Any): Any = { + val buffer = leftInput.asInstanceOf[Array[Byte]] + val memory = Memory.wrap(buffer) + + right.eval() match { + case null => null + case num: Double => + // Single value case + kllSketchGetQuantile(memory, num) + case arrayData: ArrayData => + // Array case - use batch API for better performance + val ranks = arrayData.toDoubleArray() + val results = kllSketchGetQuantiles(memory, ranks) + new GenericArrayData(results) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Extracts a single value from the quantiles sketch representing the + desired rank given the input quantile. The desired rank can either be a single value + or an array. In the latter case, the function will return an array of results of equal + length to the input array. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_bigint(col), 3) > 0.3 FROM VALUES (1), (2), (3), (4), (5) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetRankBigint(left: Expression, right: Expression) + extends KllSketchGetRankBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_get_rank_bigint" + override def inputDataType: DataType = LongType + override def kllSketchGetRank(memory: Memory, quantile: Any): Double = { + withRankErrorHandling { + KllLongsSketch.wrap(memory).getRank(quantile.asInstanceOf[Long]) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Extracts a single value from the quantiles sketch representing the + desired rank given the input quantile. The desired rank can either be a single value + or an array. In the latter case, the function will return an array of results of equal + length to the input array. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_float(col), 3.0) > 0.3 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetRankFloat(left: Expression, right: Expression) + extends KllSketchGetRankBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_get_rank_float" + override def inputDataType: DataType = FloatType + override def kllSketchGetRank(memory: Memory, quantile: Any): Double = { + withRankErrorHandling { + KllFloatsSketch.wrap(memory).getRank(quantile.asInstanceOf[Float]) + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(left, right) - Extracts a single value from the quantiles sketch representing the + desired rank given the input quantile. The desired rank can either be a single value + or an array. In the latter case, the function will return an array of results of equal + length to the input array. + """, + examples = """ + Examples: + > SELECT _FUNC_(kll_sketch_agg_double(col), 3.0) > 0.3 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col); + true + """, + group = "misc_funcs", + since = "4.1.0") +case class KllSketchGetRankDouble(left: Expression, right: Expression) + extends KllSketchGetRankBase { + override def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Expression = + copy(left = newLeft, right = newRight) + override def prettyName: String = "kll_sketch_get_rank_double" + override def inputDataType: DataType = DoubleType + override def kllSketchGetRank(memory: Memory, quantile: Any): Double = { + withRankErrorHandling { + KllDoublesSketch.wrap(memory).getRank(quantile.asInstanceOf[Double]) + } + } +} + +/** + * This is a base class for the above expressions to reduce boilerplate. + * Each implementor is expected to define two methods, one to specify the input argument data type, + * and another to compute the rank of an input sketch buffer given the input quantile. + */ +abstract class KllSketchGetRankBase + extends BinaryExpression + with CodegenFallback + with ImplicitCastInputTypes { + /** + * Helper method to wrap rank operations with consistent error handling. + * @param operation The operation to execute + * @return The result of the operation + */ + protected def withRankErrorHandling[T](operation: => T): T = { + try { + operation + } catch { + case e: Exception => + throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + } + } + + protected def inputDataType: DataType + + /** + * This method accepts a KLL quantiles Memory segment, wraps it with the corresponding + * Kll*Sketch.wrap method, and then calls getRank on the result. + * @param memory The input KLL quantiles sketch buffer to extract the rank from + * @param quantile The input quantile to use to compute the rank + * @return The result rank + */ + protected def kllSketchGetRank(memory: Memory, quantile: Any): Double + + override def nullIntolerant: Boolean = true + override def inputTypes: Seq[AbstractDataType] = { + Seq( + BinaryType, + TypeCollection( + inputDataType, + ArrayType(inputDataType, containsNull = false))) + } + override def dataType: DataType = { + right.dataType match { + case ArrayType(_, _) => ArrayType(DoubleType, false) + case _ => DoubleType + } + } + + override def nullSafeEval(leftInput: Any, rightInput: Any): Any = { + val buffer: Array[Byte] = leftInput.asInstanceOf[Array[Byte]] + val memory: Memory = Memory.wrap(buffer) + + right.eval() match { + case null => null + case value if !value.isInstanceOf[ArrayData] => + // Single value case + kllSketchGetRank(memory, value) + case arrayData: ArrayData => + // Array case - use direct iteration to avoid multiple array allocations + val numElements = arrayData.numElements() + val results = new Array[Double](numElements) + var i = 0 + inputDataType match { + case LongType => + while (i < numElements) { + results(i) = kllSketchGetRank(memory, arrayData.getLong(i)) + i += 1 + } + case FloatType => + while (i < numElements) { + results(i) = kllSketchGetRank(memory, arrayData.getFloat(i)) + i += 1 + } + case DoubleType => + while (i < numElements) { + results(i) = kllSketchGetRank(memory, arrayData.getDouble(i)) + i += 1 + } + } + new GenericArrayData(results) + } + } +} + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index a12aa53d1f6f9..1f7d2a149a7be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -3192,4 +3192,42 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE errorClass = "THETA_LG_NOM_ENTRIES_MUST_BE_CONSTANT", messageParameters = Map("function" -> toSQLId(function))) } + + def kllSketchInvalidQuantileRangeError(function: String, quantile: Double): Throwable = { + new SparkRuntimeException( + errorClass = "KLL_SKETCH_INVALID_QUANTILE_RANGE", + messageParameters = Map( + "functionName" -> toSQLId(function), + "quantile" -> toSQLValue(quantile, DoubleType))) + } + + def kllSketchInvalidInputError(function: String, reason: String): Throwable = { + new SparkRuntimeException( + errorClass = "KLL_SKETCH_INVALID_INPUT", + messageParameters = Map( + "functionName" -> toSQLId(function), + "reason" -> reason)) + } + + def kllSketchIncompatibleMergeError(function: String, reason: String): Throwable = { + new SparkRuntimeException( + errorClass = "KLL_SKETCH_INCOMPATIBLE_MERGE", + messageParameters = Map( + "functionName" -> toSQLId(function), + "reason" -> reason)) + } + + def kllSketchKMustBeConstantError(function: String): Throwable = { + new SparkRuntimeException( + errorClass = "KLL_SKETCH_K_MUST_BE_CONSTANT", + messageParameters = Map("functionName" -> toSQLId(function))) + } + + def kllSketchKOutOfRangeError(function: String, k: Int): Throwable = { + new SparkRuntimeException( + errorClass = "KLL_SKETCH_K_OUT_OF_RANGE", + messageParameters = Map( + "functionName" -> toSQLId(function), + "k" -> toSQLValue(k, IntegerType))) + } } diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 34739ec1c44c9..7b6cbabeb1296 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -182,6 +182,21 @@ | org.apache.spark.sql.catalyst.expressions.JsonObjectKeys | json_object_keys | SELECT json_object_keys('{}') | struct> | | org.apache.spark.sql.catalyst.expressions.JsonToStructs | from_json | SELECT from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE') | struct> | | org.apache.spark.sql.catalyst.expressions.JsonTuple | json_tuple | SELECT json_tuple('{"a":1, "b":2}', 'a', 'b') | struct | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetNBigint | kll_sketch_get_n_bigint | SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col)) FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetNDouble | kll_sketch_get_n_double | SELECT kll_sketch_get_n_double(kll_sketch_agg_double(col)) FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetNFloat | kll_sketch_get_n_float | SELECT kll_sketch_get_n_float(kll_sketch_agg_float(col)) FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetQuantileBigint | kll_sketch_get_quantile_bigint | SELECT kll_sketch_get_quantile_bigint(kll_sketch_agg_bigint(col), 0.5) > 1 FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct<(kll_sketch_get_quantile_bigint(kll_sketch_agg_bigint(col), 0.5) > 1):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetQuantileDouble | kll_sketch_get_quantile_double | SELECT kll_sketch_get_quantile_double(kll_sketch_agg_double(col), 0.5) > 1 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct<(kll_sketch_get_quantile_double(kll_sketch_agg_double(col), 0.5) > 1):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetQuantileFloat | kll_sketch_get_quantile_float | SELECT kll_sketch_get_quantile_float(kll_sketch_agg_float(col), 0.5) > 1 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct<(kll_sketch_get_quantile_float(kll_sketch_agg_float(col), 0.5) > 1):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetRankBigint | kll_sketch_get_rank_bigint | SELECT kll_sketch_get_rank_bigint(kll_sketch_agg_bigint(col), 3) > 0.3 FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct<(kll_sketch_get_rank_bigint(kll_sketch_agg_bigint(col), 3) > 0.3):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetRankDouble | kll_sketch_get_rank_double | SELECT kll_sketch_get_rank_double(kll_sketch_agg_double(col), 3.0) > 0.3 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct<(kll_sketch_get_rank_double(kll_sketch_agg_double(col), 3.0) > 0.3):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchGetRankFloat | kll_sketch_get_rank_float | SELECT kll_sketch_get_rank_float(kll_sketch_agg_float(col), 3.0) > 0.3 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct<(kll_sketch_get_rank_float(kll_sketch_agg_float(col), 3.0) > 0.3):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchMergeBigint | kll_sketch_merge_bigint | SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_merge_bigint(kll_sketch_agg_bigint(col), kll_sketch_agg_bigint(col)))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct<(length(kll_sketch_to_string_bigint(kll_sketch_merge_bigint(kll_sketch_agg_bigint(col), kll_sketch_agg_bigint(col)))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchMergeDouble | kll_sketch_merge_double | SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_merge_double(kll_sketch_agg_double(col), kll_sketch_agg_double(col)))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct<(length(kll_sketch_to_string_double(kll_sketch_merge_double(kll_sketch_agg_double(col), kll_sketch_agg_double(col)))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchMergeFloat | kll_sketch_merge_float | SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_merge_float(kll_sketch_agg_float(col), kll_sketch_agg_float(col)))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct<(length(kll_sketch_to_string_float(kll_sketch_merge_float(kll_sketch_agg_float(col), kll_sketch_agg_float(col)))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchToStringBigint | kll_sketch_to_string_bigint | SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct<(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchToStringDouble | kll_sketch_to_string_double | SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_agg_double(col))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct<(length(kll_sketch_to_string_double(kll_sketch_agg_double(col))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.KllSketchToStringFloat | kll_sketch_to_string_float | SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_agg_float(col))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct<(length(kll_sketch_to_string_float(kll_sketch_agg_float(col))) > 0):boolean> | | org.apache.spark.sql.catalyst.expressions.LPadExpressionBuilder | lpad | SELECT lpad('hi', 5, '??') | struct | | org.apache.spark.sql.catalyst.expressions.Lag | lag | SELECT a, b, lag(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.LastDay | last_day | SELECT last_day('2009-01-12') | struct | @@ -440,6 +455,9 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg | hll_sketch_agg | SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HllUnionAgg | hll_union_agg | SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) FROM (SELECT hll_sketch_agg(col) as sketch FROM VALUES (1) tab(col) UNION ALL SELECT hll_sketch_agg(col, 20) as sketch FROM VALUES (1) tab(col)) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.KllSketchAggBigint | kll_sketch_agg_bigint | SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct<(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.aggregate.KllSketchAggDouble | kll_sketch_agg_double | SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_agg_double(col))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct<(length(kll_sketch_to_string_double(kll_sketch_agg_double(col))) > 0):boolean> | +| org.apache.spark.sql.catalyst.expressions.aggregate.KllSketchAggFloat | kll_sketch_agg_float | SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_agg_float(col))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct<(length(kll_sketch_to_string_float(kll_sketch_agg_float(col))) > 0):boolean> | | org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out new file mode 100644 index 0000000000000..049cd163f628c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out @@ -0,0 +1,1124 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t_int_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_int_1_5_through_7_11 + + +-- !query +CREATE TABLE t_int_1_5_through_7_11 AS +VALUES + (1, 5), (2, 6), (3, 7), (4, 8), (5, 9), (6, 10), (7, 11) AS tab(col1, col2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_int_1_5_through_7_11`, ErrorIfExists, [col1, col2] + +- SubqueryAlias tab + +- LocalRelation [col1#x, col2#x] + + +-- !query +DROP TABLE IF EXISTS t_long_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_long_1_5_through_7_11 + + +-- !query +CREATE TABLE t_long_1_5_through_7_11 AS +VALUES + (1L, 5L), (2L, 6L), (3L, 7L), (4L, 8L), (5L, 9L), (6L, 10L), (7L, 11L) AS tab(col1, col2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_long_1_5_through_7_11`, ErrorIfExists, [col1, col2] + +- SubqueryAlias tab + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +DROP TABLE IF EXISTS t_short_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_short_1_5_through_7_11 + + +-- !query +CREATE TABLE t_short_1_5_through_7_11 AS +VALUES + (CAST(1 AS SMALLINT), CAST(5 AS SMALLINT)), + (CAST(2 AS SMALLINT), CAST(6 AS SMALLINT)), + (CAST(3 AS SMALLINT), CAST(7 AS SMALLINT)), + (CAST(4 AS SMALLINT), CAST(8 AS SMALLINT)), + (CAST(5 AS SMALLINT), CAST(9 AS SMALLINT)), + (CAST(6 AS SMALLINT), CAST(10 AS SMALLINT)), + (CAST(7 AS SMALLINT), CAST(11 AS SMALLINT)) + AS tab(col1, col2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_short_1_5_through_7_11`, ErrorIfExists, [col1, col2] + +- SubqueryAlias tab + +- LocalRelation [col1#x, col2#x] + + +-- !query +DROP TABLE IF EXISTS t_byte_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_byte_1_5_through_7_11 + + +-- !query +CREATE TABLE t_byte_1_5_through_7_11 AS +VALUES + (CAST(1 AS TINYINT), CAST(5 AS TINYINT)), + (CAST(2 AS TINYINT), CAST(6 AS TINYINT)), + (CAST(3 AS TINYINT), CAST(7 AS TINYINT)), + (CAST(4 AS TINYINT), CAST(8 AS TINYINT)), + (CAST(5 AS TINYINT), CAST(9 AS TINYINT)), + (CAST(6 AS TINYINT), CAST(10 AS TINYINT)), + (CAST(7 AS TINYINT), CAST(11 AS TINYINT)) + AS tab(col1, col2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_byte_1_5_through_7_11`, ErrorIfExists, [col1, col2] + +- SubqueryAlias tab + +- LocalRelation [col1#x, col2#x] + + +-- !query +DROP TABLE IF EXISTS t_float_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_float_1_5_through_7_11 + + +-- !query +CREATE TABLE t_float_1_5_through_7_11 AS +VALUES + (CAST(1 AS FLOAT), CAST(5 AS FLOAT)), + (CAST(2 AS FLOAT), CAST(6 AS FLOAT)), + (CAST(3 AS FLOAT), CAST(7 AS FLOAT)), + (CAST(4 AS FLOAT), CAST(8 AS FLOAT)), + (CAST(5 AS FLOAT), CAST(9 AS FLOAT)), + (CAST(6 AS FLOAT), CAST(10 AS FLOAT)), + (CAST(7 AS FLOAT), CAST(11 AS FLOAT)) AS tab(col1, col2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_float_1_5_through_7_11`, ErrorIfExists, [col1, col2] + +- SubqueryAlias tab + +- LocalRelation [col1#x, col2#x] + + +-- !query +DROP TABLE IF EXISTS t_double_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_double_1_5_through_7_11 + + +-- !query +CREATE TABLE t_double_1_5_through_7_11 AS +VALUES + (CAST(1 AS DOUBLE), CAST(5 AS DOUBLE)), + (CAST(2 AS DOUBLE), CAST(6 AS DOUBLE)), + (CAST(3 AS DOUBLE), CAST(7 AS DOUBLE)), + (CAST(4 AS DOUBLE), CAST(8 AS DOUBLE)), + (CAST(5 AS DOUBLE), CAST(9 AS DOUBLE)), + (CAST(6 AS DOUBLE), CAST(10 AS DOUBLE)), + (CAST(7 AS DOUBLE), CAST(11 AS DOUBLE)) AS tab(col1, col2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_double_1_5_through_7_11`, ErrorIfExists, [col1, col2] + +- SubqueryAlias tab + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_byte_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_bigint(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 + +- Relation spark_catalog.default.t_byte_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_int_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_bigint(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_int_1_5_through_7_11 + +- Relation spark_catalog.default.t_int_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_bigint(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_short_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_bigint(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_short_1_5_through_7_11 + +- Relation spark_catalog.default.t_short_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_float(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((cast(kll_sketch_get_quantile_float(agg#x, cast(0.5 as double)) as double) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_float(agg#x, cast(3 as float)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_double(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_double(agg#x, cast(3 as double)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 + +- Relation spark_catalog.default.t_double_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +Project [split(kll_sketch_to_string_double(agg#x), +, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_double(agg#x, cast(3 as double)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT + split( + kll_sketch_to_string_bigint( + kll_sketch_merge_bigint( + kll_sketch_agg_bigint(col1), + kll_sketch_agg_bigint(col1) + ) + ), + '\n' + )[1] AS result + FROM t_byte_1_5_through_7_11 +-- !query analysis +Aggregate [split(kll_sketch_to_string_bigint(kll_sketch_merge_bigint(kll_sketch_agg_bigint(col1#x, None, 0, 0), kll_sketch_agg_bigint(col1#x, None, 0, 0))), +, -1)[1] AS result#x] ++- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 + +- Relation spark_catalog.default.t_byte_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT + split( + kll_sketch_to_string_float( + kll_sketch_merge_float( + kll_sketch_agg_float(col1), + kll_sketch_agg_float(col1) + ) + ), + '\n' + )[1] AS result +FROM t_byte_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 87, + "stopIndex" : 112, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT + split( + kll_sketch_to_string_double( + kll_sketch_merge_double( + kll_sketch_agg_double(col1), + kll_sketch_agg_double(col1) + ) + ), + '\n' + )[1] AS result +FROM t_byte_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 89, + "stopIndex" : 115, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match, + abs(kll_sketch_get_rank_bigint(agg_with_nulls, 4) - + kll_sketch_get_rank_bigint(agg_without_nulls, 4)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg_with_nulls + FROM (VALUES (1L), (CAST(NULL AS BIGINT)), (3L), (5L), (CAST(NULL AS BIGINT)), (7L)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_bigint(col1) AS agg_without_nulls + FROM (VALUES (1L), (3L), (5L), (7L)) AS tab(col1) +) WITHOUT_NULLS +-- !query analysis +Project [(abs((kll_sketch_get_quantile_bigint(agg_with_nulls#x, cast(0.5 as double)) - kll_sketch_get_quantile_bigint(agg_without_nulls#x, cast(0.5 as double)))) < cast(1 as bigint)) AS medians_match#x, (abs((kll_sketch_get_rank_bigint(agg_with_nulls#x, cast(4 as bigint)) - kll_sketch_get_rank_bigint(agg_without_nulls#x, cast(4 as bigint)))) < cast(0.1 as double)) AS ranks_match#x] ++- Join Inner + :- SubqueryAlias WITH_NULLS + : +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg_with_nulls#x] + : +- SubqueryAlias tab + : +- Project [col1#xL AS col1#xL] + : +- LocalRelation [col1#xL] + +- SubqueryAlias WITHOUT_NULLS + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg_without_nulls#x] + +- SubqueryAlias tab + +- Project [col1#xL AS col1#xL] + +- LocalRelation [col1#xL] + + +-- !query +SELECT abs(kll_sketch_get_quantile_float(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_float(agg_without_nulls, 0.5)) < 0.5 AS medians_match, + abs(kll_sketch_get_rank_float(agg_with_nulls, 4.0) - + kll_sketch_get_rank_float(agg_without_nulls, 4.0)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg_with_nulls + FROM (VALUES (1.0F), (CAST(NULL AS FLOAT)), (3.0F), (5.0F), (CAST(NULL AS FLOAT)), (7.0F)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_float(col1) AS agg_without_nulls + FROM (VALUES (1.0F), (3.0F), (5.0F), (7.0F)) AS tab(col1) +) WITHOUT_NULLS +-- !query analysis +Project [(cast(abs((kll_sketch_get_quantile_float(agg_with_nulls#x, cast(0.5 as double)) - kll_sketch_get_quantile_float(agg_without_nulls#x, cast(0.5 as double)))) as double) < cast(0.5 as double)) AS medians_match#x, (abs((kll_sketch_get_rank_float(agg_with_nulls#x, cast(4.0 as float)) - kll_sketch_get_rank_float(agg_without_nulls#x, cast(4.0 as float)))) < cast(0.1 as double)) AS ranks_match#x] ++- Join Inner + :- SubqueryAlias WITH_NULLS + : +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS agg_with_nulls#x] + : +- SubqueryAlias tab + : +- Project [col1#x AS col1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias WITHOUT_NULLS + +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS agg_without_nulls#x] + +- SubqueryAlias tab + +- Project [col1#x AS col1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT abs(kll_sketch_get_quantile_double(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_double(agg_without_nulls, 0.5)) < 0.5 AS medians_match, + abs(kll_sketch_get_rank_double(agg_with_nulls, 4.0) - + kll_sketch_get_rank_double(agg_without_nulls, 4.0)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg_with_nulls + FROM (VALUES (1.0D), (CAST(NULL AS DOUBLE)), (3.0D), (5.0D), (CAST(NULL AS DOUBLE)), (7.0D)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_double(col1) AS agg_without_nulls + FROM (VALUES (1.0D), (3.0D), (5.0D), (7.0D)) AS tab(col1) +) WITHOUT_NULLS +-- !query analysis +Project [(abs((kll_sketch_get_quantile_double(agg_with_nulls#x, cast(0.5 as double)) - kll_sketch_get_quantile_double(agg_without_nulls#x, cast(0.5 as double)))) < cast(0.5 as double)) AS medians_match#x, (abs((kll_sketch_get_rank_double(agg_with_nulls#x, cast(4.0 as double)) - kll_sketch_get_rank_double(agg_without_nulls#x, cast(4.0 as double)))) < cast(0.1 as double)) AS ranks_match#x] ++- Join Inner + :- SubqueryAlias WITH_NULLS + : +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS agg_with_nulls#x] + : +- SubqueryAlias tab + : +- Project [col1#x AS col1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias WITHOUT_NULLS + +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS agg_without_nulls#x] + +- SubqueryAlias tab + +- Project [col1#x AS col1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT kll_sketch_get_quantile_bigint(CAST(NULL AS BINARY), 0.5) AS null_sketch +-- !query analysis +Project [kll_sketch_get_quantile_bigint(cast(null as binary), cast(0.5 as double)) AS null_sketch#xL] ++- OneRowRelation + + +-- !query +SELECT kll_sketch_get_rank_float(CAST(NULL AS BINARY), 5.0) AS null_sketch +-- !query analysis +Project [kll_sketch_get_rank_float(cast(null as binary), cast(5.0 as float)) AS null_sketch#x] ++- OneRowRelation + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 8))) > 0 AS k_min_value +FROM t_long_1_5_through_7_11 +-- !query analysis +Aggregate [(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1#xL, Some(8), 0, 0))) > 0) AS k_min_value#x] ++- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 200))) > 0 AS k_default_value +FROM t_long_1_5_through_7_11 +-- !query analysis +Aggregate [(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1#xL, Some(200), 0, 0))) > 0) AS k_default_value#x] ++- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 400))) > 0 AS k_custom_value +FROM t_long_1_5_through_7_11 +-- !query analysis +Aggregate [(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1#xL, Some(400), 0, 0))) > 0) AS k_custom_value#x] ++- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 65535))) > 0 AS k_max_value +FROM t_long_1_5_through_7_11 +-- !query analysis +Aggregate [(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1#xL, Some(65535), 0, 0))) > 0) AS k_max_value#x] ++- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_agg_float(col1, 100))) > 0 AS k_float_sketch +FROM t_float_1_5_through_7_11 +-- !query analysis +Aggregate [(length(kll_sketch_to_string_float(kll_sketch_agg_float(col1#x, Some(100), 0, 0))) > 0) AS k_float_sketch#x] ++- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_agg_double(col1, 300))) > 0 AS k_double_sketch +FROM t_double_1_5_through_7_11 +-- !query analysis +Aggregate [(length(kll_sketch_to_string_double(kll_sketch_agg_double(col1#x, Some(300), 0, 0))) > 0) AS k_double_sketch#x] ++- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 + +- Relation spark_catalog.default.t_double_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_bigint +FROM t_long_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1#xL, None, 0, 0)) AS n_bigint#xL] ++- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_byte +FROM t_byte_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1#x, None, 0, 0)) AS n_byte#xL] ++- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 + +- Relation spark_catalog.default.t_byte_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_short +FROM t_short_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1#x, None, 0, 0)) AS n_short#xL] ++- SubqueryAlias spark_catalog.default.t_short_1_5_through_7_11 + +- Relation spark_catalog.default.t_short_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_int +FROM t_int_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1#x, None, 0, 0)) AS n_int#xL] ++- SubqueryAlias spark_catalog.default.t_int_1_5_through_7_11 + +- Relation spark_catalog.default.t_int_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_float(kll_sketch_agg_float(col1)) AS n_float +FROM t_float_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_float(kll_sketch_agg_float(col1#x, None, 0, 0)) AS n_float#xL] ++- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_double(kll_sketch_agg_double(col1)) AS n_double +FROM t_double_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_double(kll_sketch_agg_double(col1#x, None, 0, 0)) AS n_double#xL] ++- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 + +- Relation spark_catalog.default.t_double_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1, 100)) AS n_k_100 +FROM t_long_1_5_through_7_11 +-- !query analysis +Aggregate [kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1#xL, Some(100), 0, 0)) AS n_k_100#xL] ++- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "(\"TINYINT\" or \"INT\" or \"BIGINT\" or \"SMALLINT\")", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 273, + "stopIndex" : 299, + "fragment" : "kll_sketch_agg_bigint(col1)" + } ] +} + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "(\"TINYINT\" or \"INT\" or \"BIGINT\" or \"SMALLINT\")", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 273, + "stopIndex" : 299, + "fragment" : "kll_sketch_agg_bigint(col1)" + } ] +} + + +-- !query +SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 274, + "stopIndex" : 299, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_bigint +FROM t_long_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_int +FROM t_int_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_short +FROM t_short_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_byte +FROM t_byte_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_bigint +FROM t_long_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_int +FROM t_int_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_short +FROM t_short_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_byte +FROM t_byte_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, -0.5) AS invalid_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_get_quantile_bigint(agg#x, cast(-0.5 as double)) AS invalid_quantile#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, 1.5) AS invalid_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_get_quantile_bigint(agg#x, cast(1.5 as double)) AS invalid_quantile#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_get_quantile_float(agg, array(-0.1, 0.5, 1.5)) AS invalid_quantiles +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_get_quantile_float(agg#x, cast(array(cast(-0.1 as decimal(2,1)), cast(0.5 as decimal(2,1)), 1.5) as array)) AS invalid_quantiles#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, 5) AS wrong_type +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_get_rank_bigint(agg#x, cast(5 as bigint)) AS wrong_type#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_merge_bigint(agg1, agg2) AS incompatible_merge +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg1, + kll_sketch_agg_float(CAST(col1 AS FLOAT)) AS agg2 + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_merge_bigint(agg1#x, agg2#x) AS incompatible_merge#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg1#x, kll_sketch_agg_float(cast(col1#xL as float), None, 0, 0) AS agg2#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_get_quantile_bigint(CAST('not_a_sketch' AS BINARY), 0.5) AS invalid_binary +-- !query analysis +Project [kll_sketch_get_quantile_bigint(cast(not_a_sketch as binary), cast(0.5 as double)) AS invalid_binary#xL] ++- OneRowRelation + + +-- !query +SELECT kll_sketch_get_quantile_float(agg, 0.5) IS NOT NULL AS returns_value +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [isnotnull(kll_sketch_get_quantile_float(agg#x, cast(0.5 as double))) AS returns_value#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_to_string_double(agg) LIKE '%Kll%' AS contains_kll_header +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_to_string_double(agg#x) LIKE %Kll% AS contains_kll_header#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_agg_bigint(col1, 7) AS k_too_small +FROM t_long_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_sketch_agg_bigint`, the k parameter must be between 8 and 65535 (inclusive), but got 7. SQLSTATE: 22003", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1, 7)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "kll_sketch_agg_bigint(col1, 7)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_bigint(col1, 65536) AS k_too_large +FROM t_long_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_sketch_agg_bigint`, the k parameter must be between 8 and 65535 (inclusive), but got 65536. SQLSTATE: 22003", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1, 65536)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "kll_sketch_agg_bigint(col1, 65536)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1, CAST(NULL AS INT)) AS k_is_null +FROM t_float_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "k", + "sqlExpr" : "\"kll_sketch_agg_float(col1, CAST(NULL AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "kll_sketch_agg_float(col1, CAST(NULL AS INT))" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1, CAST(col1 AS INT)) AS k_non_constant +FROM t_double_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "CAST(spark_catalog.default.t_double_1_5_through_7_11.col1 AS INT)", + "inputName" : "k", + "inputType" : "int", + "sqlExpr" : "\"kll_sketch_agg_double(col1, CAST(col1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "kll_sketch_agg_double(col1, CAST(col1 AS INT))" + } ] +} + + +-- !query +SELECT kll_sketch_agg_bigint(col1, '100') AS k_wrong_type +FROM t_long_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"100\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"INT\"", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1, 100)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "kll_sketch_agg_bigint(col1, '100')" + } ] +} + + +-- !query +SELECT kll_sketch_get_n_bigint(X'deadbeef') AS invalid_binary_bigint +-- !query analysis +Project [kll_sketch_get_n_bigint(0xDEADBEEF) AS invalid_binary_bigint#xL] ++- OneRowRelation + + +-- !query +SELECT kll_sketch_get_n_float(X'cafebabe') AS invalid_binary_float +-- !query analysis +Project [kll_sketch_get_n_float(0xCAFEBABE) AS invalid_binary_float#xL] ++- OneRowRelation + + +-- !query +SELECT kll_sketch_get_n_double(X'12345678') AS invalid_binary_double +-- !query analysis +Project [kll_sketch_get_n_double(0x12345678) AS invalid_binary_double#xL] ++- OneRowRelation + + +-- !query +DROP TABLE IF EXISTS t_int_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_int_1_5_through_7_11 + + +-- !query +DROP TABLE IF EXISTS t_long_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_long_1_5_through_7_11 + + +-- !query +DROP TABLE IF EXISTS t_short_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_short_1_5_through_7_11 + + +-- !query +DROP TABLE IF EXISTS t_byte_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_byte_1_5_through_7_11 + + +-- !query +DROP TABLE IF EXISTS t_float_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_float_1_5_through_7_11 + + +-- !query +DROP TABLE IF EXISTS t_double_1_5_through_7_11 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t_double_1_5_through_7_11 diff --git a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql new file mode 100644 index 0000000000000..404235b33e670 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql @@ -0,0 +1,422 @@ +-- Positive test cases +-- Create tables with two columns for each data type + +-- Integer table +DROP TABLE IF EXISTS t_int_1_5_through_7_11; +CREATE TABLE t_int_1_5_through_7_11 AS +VALUES + (1, 5), (2, 6), (3, 7), (4, 8), (5, 9), (6, 10), (7, 11) AS tab(col1, col2); + +-- Long table +DROP TABLE IF EXISTS t_long_1_5_through_7_11; +CREATE TABLE t_long_1_5_through_7_11 AS +VALUES + (1L, 5L), (2L, 6L), (3L, 7L), (4L, 8L), (5L, 9L), (6L, 10L), (7L, 11L) AS tab(col1, col2); + +-- SMALLINT (ShortType) table +DROP TABLE IF EXISTS t_short_1_5_through_7_11; +CREATE TABLE t_short_1_5_through_7_11 AS +VALUES + (CAST(1 AS SMALLINT), CAST(5 AS SMALLINT)), + (CAST(2 AS SMALLINT), CAST(6 AS SMALLINT)), + (CAST(3 AS SMALLINT), CAST(7 AS SMALLINT)), + (CAST(4 AS SMALLINT), CAST(8 AS SMALLINT)), + (CAST(5 AS SMALLINT), CAST(9 AS SMALLINT)), + (CAST(6 AS SMALLINT), CAST(10 AS SMALLINT)), + (CAST(7 AS SMALLINT), CAST(11 AS SMALLINT)) + AS tab(col1, col2); + +-- TINYINT (ByteType) table +DROP TABLE IF EXISTS t_byte_1_5_through_7_11; +CREATE TABLE t_byte_1_5_through_7_11 AS +VALUES + (CAST(1 AS TINYINT), CAST(5 AS TINYINT)), + (CAST(2 AS TINYINT), CAST(6 AS TINYINT)), + (CAST(3 AS TINYINT), CAST(7 AS TINYINT)), + (CAST(4 AS TINYINT), CAST(8 AS TINYINT)), + (CAST(5 AS TINYINT), CAST(9 AS TINYINT)), + (CAST(6 AS TINYINT), CAST(10 AS TINYINT)), + (CAST(7 AS TINYINT), CAST(11 AS TINYINT)) + AS tab(col1, col2); + +-- Float table +DROP TABLE IF EXISTS t_float_1_5_through_7_11; +CREATE TABLE t_float_1_5_through_7_11 AS +VALUES + (CAST(1 AS FLOAT), CAST(5 AS FLOAT)), + (CAST(2 AS FLOAT), CAST(6 AS FLOAT)), + (CAST(3 AS FLOAT), CAST(7 AS FLOAT)), + (CAST(4 AS FLOAT), CAST(8 AS FLOAT)), + (CAST(5 AS FLOAT), CAST(9 AS FLOAT)), + (CAST(6 AS FLOAT), CAST(10 AS FLOAT)), + (CAST(7 AS FLOAT), CAST(11 AS FLOAT)) AS tab(col1, col2); + +-- Double table +DROP TABLE IF EXISTS t_double_1_5_through_7_11; +CREATE TABLE t_double_1_5_through_7_11 AS +VALUES + (CAST(1 AS DOUBLE), CAST(5 AS DOUBLE)), + (CAST(2 AS DOUBLE), CAST(6 AS DOUBLE)), + (CAST(3 AS DOUBLE), CAST(7 AS DOUBLE)), + (CAST(4 AS DOUBLE), CAST(8 AS DOUBLE)), + (CAST(5 AS DOUBLE), CAST(9 AS DOUBLE)), + (CAST(6 AS DOUBLE), CAST(10 AS DOUBLE)), + (CAST(7 AS DOUBLE), CAST(11 AS DOUBLE)) AS tab(col1, col2); + +-- BIGINT sketches +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_byte_1_5_through_7_11 +); + +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_int_1_5_through_7_11 +); + +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_short_1_5_through_7_11 +); + +-- FLOAT sketches (only accepts float types to avoid precision loss) +SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- DOUBLE sketches (accepts float and double types to avoid precision loss from integer conversion) +SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +); + +-- Test float column with double sketch (valid type promotion) +SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- Merging sketches and converting them to strings +SELECT + split( + kll_sketch_to_string_bigint( + kll_sketch_merge_bigint( + kll_sketch_agg_bigint(col1), + kll_sketch_agg_bigint(col1) + ) + ), + '\n' + )[1] AS result + FROM t_byte_1_5_through_7_11; + +SELECT + split( + kll_sketch_to_string_float( + kll_sketch_merge_float( + kll_sketch_agg_float(col1), + kll_sketch_agg_float(col1) + ) + ), + '\n' + )[1] AS result +FROM t_byte_1_5_through_7_11; + +SELECT + split( + kll_sketch_to_string_double( + kll_sketch_merge_double( + kll_sketch_agg_double(col1), + kll_sketch_agg_double(col1) + ) + ), + '\n' + )[1] AS result +FROM t_byte_1_5_through_7_11; + +-- Tests verifying that NULL input values are ignored by aggregate functions + +-- Test BIGINT aggregate ignores NULL values +-- Verify that the sketch computed with NULLs matches the sketch without NULLs +-- Both should compute median of [1, 3, 5, 7] which is 4 +-- Input data: 1, NULL, 3, 5, NULL, 7 +SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match, + abs(kll_sketch_get_rank_bigint(agg_with_nulls, 4) - + kll_sketch_get_rank_bigint(agg_without_nulls, 4)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg_with_nulls + FROM (VALUES (1L), (CAST(NULL AS BIGINT)), (3L), (5L), (CAST(NULL AS BIGINT)), (7L)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_bigint(col1) AS agg_without_nulls + FROM (VALUES (1L), (3L), (5L), (7L)) AS tab(col1) +) WITHOUT_NULLS; + +-- Test FLOAT aggregate ignores NULL values +-- Verify that the sketch computed with NULLs matches the sketch without NULLs +-- Input data: 1.0, NULL, 3.0, 5.0, NULL, 7.0 +SELECT abs(kll_sketch_get_quantile_float(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_float(agg_without_nulls, 0.5)) < 0.5 AS medians_match, + abs(kll_sketch_get_rank_float(agg_with_nulls, 4.0) - + kll_sketch_get_rank_float(agg_without_nulls, 4.0)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg_with_nulls + FROM (VALUES (1.0F), (CAST(NULL AS FLOAT)), (3.0F), (5.0F), (CAST(NULL AS FLOAT)), (7.0F)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_float(col1) AS agg_without_nulls + FROM (VALUES (1.0F), (3.0F), (5.0F), (7.0F)) AS tab(col1) +) WITHOUT_NULLS; + +-- Test DOUBLE aggregate ignores NULL values +-- Verify that the sketch computed with NULLs matches the sketch without NULLs +-- Input data: 1.0, NULL, 3.0, 5.0, NULL, 7.0 +SELECT abs(kll_sketch_get_quantile_double(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_double(agg_without_nulls, 0.5)) < 0.5 AS medians_match, + abs(kll_sketch_get_rank_double(agg_with_nulls, 4.0) - + kll_sketch_get_rank_double(agg_without_nulls, 4.0)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg_with_nulls + FROM (VALUES (1.0D), (CAST(NULL AS DOUBLE)), (3.0D), (5.0D), (CAST(NULL AS DOUBLE)), (7.0D)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_double(col1) AS agg_without_nulls + FROM (VALUES (1.0D), (3.0D), (5.0D), (7.0D)) AS tab(col1) +) WITHOUT_NULLS; + +-- Tests covering NULLs +-- NULL sketch to get_quantile +SELECT kll_sketch_get_quantile_bigint(CAST(NULL AS BINARY), 0.5) AS null_sketch; + +-- NULL sketch to get_rank +SELECT kll_sketch_get_rank_float(CAST(NULL AS BINARY), 5.0) AS null_sketch; + +-- Tests for the optional k parameter +-- Positive tests with valid k values +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 8))) > 0 AS k_min_value +FROM t_long_1_5_through_7_11; + +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 200))) > 0 AS k_default_value +FROM t_long_1_5_through_7_11; + +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 400))) > 0 AS k_custom_value +FROM t_long_1_5_through_7_11; + +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 65535))) > 0 AS k_max_value +FROM t_long_1_5_through_7_11; + +SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_agg_float(col1, 100))) > 0 AS k_float_sketch +FROM t_float_1_5_through_7_11; + +SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_agg_double(col1, 300))) > 0 AS k_double_sketch +FROM t_double_1_5_through_7_11; + +-- Tests for kll_sketch_get_n functions +-- BIGINT sketches +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_bigint +FROM t_long_1_5_through_7_11; + +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_byte +FROM t_byte_1_5_through_7_11; + +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_short +FROM t_short_1_5_through_7_11; + +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_int +FROM t_int_1_5_through_7_11; + +-- FLOAT sketches +SELECT kll_sketch_get_n_float(kll_sketch_agg_float(col1)) AS n_float +FROM t_float_1_5_through_7_11; + +-- DOUBLE sketches +SELECT kll_sketch_get_n_double(kll_sketch_agg_double(col1)) AS n_double +FROM t_double_1_5_through_7_11; + +-- Test with different k values +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1, 100)) AS n_k_100 +FROM t_long_1_5_through_7_11; + +-- Negative tests +-- These queries should fail with type mismatch or validation errors + +-- Type mismatch: BIGINT sketch does not accept DOUBLE columns +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_double_1_5_through_7_11 +); + +-- Type mismatch: BIGINT sketch does not accept FLOAT columns +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- Type mismatch: FLOAT sketch does not accept DOUBLE columns +SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_double_1_5_through_7_11 +); + +-- Type mismatch: FLOAT sketch does not accept integer types (BIGINT) to avoid precision loss +SELECT kll_sketch_agg_float(col1) AS invalid_float_bigint +FROM t_long_1_5_through_7_11; + +-- Type mismatch: FLOAT sketch does not accept integer types (INT) to avoid precision loss +SELECT kll_sketch_agg_float(col1) AS invalid_float_int +FROM t_int_1_5_through_7_11; + +-- Type mismatch: FLOAT sketch does not accept integer types (SMALLINT) to avoid precision loss +SELECT kll_sketch_agg_float(col1) AS invalid_float_short +FROM t_short_1_5_through_7_11; + +-- Type mismatch: FLOAT sketch does not accept integer types (TINYINT) to avoid precision loss +SELECT kll_sketch_agg_float(col1) AS invalid_float_byte +FROM t_byte_1_5_through_7_11; + +-- Type mismatch: DOUBLE sketch does not accept integer types (BIGINT) to avoid precision loss +SELECT kll_sketch_agg_double(col1) AS invalid_double_bigint +FROM t_long_1_5_through_7_11; + +-- Type mismatch: DOUBLE sketch does not accept integer types (INT) to avoid precision loss +SELECT kll_sketch_agg_double(col1) AS invalid_double_int +FROM t_int_1_5_through_7_11; + +-- Type mismatch: DOUBLE sketch does not accept integer types (SMALLINT) to avoid precision loss +SELECT kll_sketch_agg_double(col1) AS invalid_double_short +FROM t_short_1_5_through_7_11; + +-- Type mismatch: DOUBLE sketch does not accept integer types (TINYINT) to avoid precision loss +SELECT kll_sketch_agg_double(col1) AS invalid_double_byte +FROM t_byte_1_5_through_7_11; + +-- Invalid quantile: quantile value must be between 0 and 1 (negative value) +SELECT kll_sketch_get_quantile_bigint(agg, -0.5) AS invalid_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +-- Invalid quantile: quantile value must be between 0 and 1 (value > 1) +SELECT kll_sketch_get_quantile_bigint(agg, 1.5) AS invalid_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +-- Invalid quantile: quantile array with out of range values +SELECT kll_sketch_get_quantile_float(agg, array(-0.1, 0.5, 1.5)) AS invalid_quantiles +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- Type mismatch: wrong sketch type for get_rank function +SELECT kll_sketch_get_rank_bigint(agg, 5) AS wrong_type +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- Type mismatch: incompatible sketches in merge (BIGINT and FLOAT) +SELECT kll_sketch_merge_bigint(agg1, agg2) AS incompatible_merge +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg1, + kll_sketch_agg_float(CAST(col1 AS FLOAT)) AS agg2 + FROM t_long_1_5_through_7_11 +); + +-- Invalid input: non-sketch binary data to get_quantile +SELECT kll_sketch_get_quantile_bigint(CAST('not_a_sketch' AS BINARY), 0.5) AS invalid_binary; + +-- Note: get_quantile functions cannot detect sketch type mismatches at the binary level. +-- This query succeeds even though we're using a FLOAT get_quantile on a BIGINT sketch, +-- but it returns garbage values because it interprets the BIGINT binary data as FLOAT data. +SELECT kll_sketch_get_quantile_float(agg, 0.5) IS NOT NULL AS returns_value +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +-- Note: to_string functions cannot detect sketch type mismatches because they just +-- interpret the binary data. This query succeeds even though we're using a DOUBLE +-- to_string function on a BIGINT sketch. The function reads the binary representation +-- and produces output, but the numeric values will be incorrectly interpreted. +SELECT kll_sketch_to_string_double(agg) LIKE '%Kll%' AS contains_kll_header +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +-- Negative tests for k parameter +-- k parameter too small (minimum is 8) +SELECT kll_sketch_agg_bigint(col1, 7) AS k_too_small +FROM t_long_1_5_through_7_11; + +-- k parameter too large (maximum is 65535) +SELECT kll_sketch_agg_bigint(col1, 65536) AS k_too_large +FROM t_long_1_5_through_7_11; + +-- k parameter is NULL +SELECT kll_sketch_agg_float(col1, CAST(NULL AS INT)) AS k_is_null +FROM t_float_1_5_through_7_11; + +-- k parameter is not foldable (non-constant) +SELECT kll_sketch_agg_double(col1, CAST(col1 AS INT)) AS k_non_constant +FROM t_double_1_5_through_7_11; + +-- k parameter has wrong type (STRING instead of INT) +SELECT kll_sketch_agg_bigint(col1, '100') AS k_wrong_type +FROM t_long_1_5_through_7_11; + +-- Negative tests for kll_sketch_get_n functions +-- Invalid binary data +SELECT kll_sketch_get_n_bigint(X'deadbeef') AS invalid_binary_bigint; + +SELECT kll_sketch_get_n_float(X'cafebabe') AS invalid_binary_float; + +SELECT kll_sketch_get_n_double(X'12345678') AS invalid_binary_double; + +-- Clean up +DROP TABLE IF EXISTS t_int_1_5_through_7_11; +DROP TABLE IF EXISTS t_long_1_5_through_7_11; +DROP TABLE IF EXISTS t_short_1_5_through_7_11; +DROP TABLE IF EXISTS t_byte_1_5_through_7_11; +DROP TABLE IF EXISTS t_float_1_5_through_7_11; +DROP TABLE IF EXISTS t_double_1_5_through_7_11; diff --git a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out new file mode 100644 index 0000000000000..383a8e66fce3c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out @@ -0,0 +1,1187 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t_int_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t_int_1_5_through_7_11 AS +VALUES + (1, 5), (2, 6), (3, 7), (4, 8), (5, 9), (6, 10), (7, 11) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t_long_1_5_through_7_11 AS +VALUES + (1L, 5L), (2L, 6L), (3L, 7L), (4L, 8L), (5L, 9L), (6L, 10L), (7L, 11L) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_short_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t_short_1_5_through_7_11 AS +VALUES + (CAST(1 AS SMALLINT), CAST(5 AS SMALLINT)), + (CAST(2 AS SMALLINT), CAST(6 AS SMALLINT)), + (CAST(3 AS SMALLINT), CAST(7 AS SMALLINT)), + (CAST(4 AS SMALLINT), CAST(8 AS SMALLINT)), + (CAST(5 AS SMALLINT), CAST(9 AS SMALLINT)), + (CAST(6 AS SMALLINT), CAST(10 AS SMALLINT)), + (CAST(7 AS SMALLINT), CAST(11 AS SMALLINT)) + AS tab(col1, col2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_byte_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t_byte_1_5_through_7_11 AS +VALUES + (CAST(1 AS TINYINT), CAST(5 AS TINYINT)), + (CAST(2 AS TINYINT), CAST(6 AS TINYINT)), + (CAST(3 AS TINYINT), CAST(7 AS TINYINT)), + (CAST(4 AS TINYINT), CAST(8 AS TINYINT)), + (CAST(5 AS TINYINT), CAST(9 AS TINYINT)), + (CAST(6 AS TINYINT), CAST(10 AS TINYINT)), + (CAST(7 AS TINYINT), CAST(11 AS TINYINT)) + AS tab(col1, col2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_float_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t_float_1_5_through_7_11 AS +VALUES + (CAST(1 AS FLOAT), CAST(5 AS FLOAT)), + (CAST(2 AS FLOAT), CAST(6 AS FLOAT)), + (CAST(3 AS FLOAT), CAST(7 AS FLOAT)), + (CAST(4 AS FLOAT), CAST(8 AS FLOAT)), + (CAST(5 AS FLOAT), CAST(9 AS FLOAT)), + (CAST(6 AS FLOAT), CAST(10 AS FLOAT)), + (CAST(7 AS FLOAT), CAST(11 AS FLOAT)) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_double_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t_double_1_5_through_7_11 AS +VALUES + (CAST(1 AS DOUBLE), CAST(5 AS DOUBLE)), + (CAST(2 AS DOUBLE), CAST(6 AS DOUBLE)), + (CAST(3 AS DOUBLE), CAST(7 AS DOUBLE)), + (CAST(4 AS DOUBLE), CAST(8 AS DOUBLE)), + (CAST(5 AS DOUBLE), CAST(9 AS DOUBLE)), + (CAST(6 AS DOUBLE), CAST(10 AS DOUBLE)), + (CAST(7 AS DOUBLE), CAST(11 AS DOUBLE)) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_byte_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_int_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_short_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT + split( + kll_sketch_to_string_bigint( + kll_sketch_merge_bigint( + kll_sketch_agg_bigint(col1), + kll_sketch_agg_bigint(col1) + ) + ), + '\n' + )[1] AS result + FROM t_byte_1_5_through_7_11 +-- !query schema +struct +-- !query output +### KllKllLongsSketch Summary: + + +-- !query +SELECT + split( + kll_sketch_to_string_float( + kll_sketch_merge_float( + kll_sketch_agg_float(col1), + kll_sketch_agg_float(col1) + ) + ), + '\n' + )[1] AS result +FROM t_byte_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 87, + "stopIndex" : 112, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT + split( + kll_sketch_to_string_double( + kll_sketch_merge_double( + kll_sketch_agg_double(col1), + kll_sketch_agg_double(col1) + ) + ), + '\n' + )[1] AS result +FROM t_byte_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 89, + "stopIndex" : 115, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match, + abs(kll_sketch_get_rank_bigint(agg_with_nulls, 4) - + kll_sketch_get_rank_bigint(agg_without_nulls, 4)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg_with_nulls + FROM (VALUES (1L), (CAST(NULL AS BIGINT)), (3L), (5L), (CAST(NULL AS BIGINT)), (7L)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_bigint(col1) AS agg_without_nulls + FROM (VALUES (1L), (3L), (5L), (7L)) AS tab(col1) +) WITHOUT_NULLS +-- !query schema +struct +-- !query output +true true + + +-- !query +SELECT abs(kll_sketch_get_quantile_float(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_float(agg_without_nulls, 0.5)) < 0.5 AS medians_match, + abs(kll_sketch_get_rank_float(agg_with_nulls, 4.0) - + kll_sketch_get_rank_float(agg_without_nulls, 4.0)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg_with_nulls + FROM (VALUES (1.0F), (CAST(NULL AS FLOAT)), (3.0F), (5.0F), (CAST(NULL AS FLOAT)), (7.0F)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_float(col1) AS agg_without_nulls + FROM (VALUES (1.0F), (3.0F), (5.0F), (7.0F)) AS tab(col1) +) WITHOUT_NULLS +-- !query schema +struct +-- !query output +true true + + +-- !query +SELECT abs(kll_sketch_get_quantile_double(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_double(agg_without_nulls, 0.5)) < 0.5 AS medians_match, + abs(kll_sketch_get_rank_double(agg_with_nulls, 4.0) - + kll_sketch_get_rank_double(agg_without_nulls, 4.0)) < 0.1 AS ranks_match +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg_with_nulls + FROM (VALUES (1.0D), (CAST(NULL AS DOUBLE)), (3.0D), (5.0D), (CAST(NULL AS DOUBLE)), (7.0D)) AS tab(col1) +) WITH_NULLS, +( + SELECT kll_sketch_agg_double(col1) AS agg_without_nulls + FROM (VALUES (1.0D), (3.0D), (5.0D), (7.0D)) AS tab(col1) +) WITHOUT_NULLS +-- !query schema +struct +-- !query output +true true + + +-- !query +SELECT kll_sketch_get_quantile_bigint(CAST(NULL AS BINARY), 0.5) AS null_sketch +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT kll_sketch_get_rank_float(CAST(NULL AS BINARY), 5.0) AS null_sketch +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 8))) > 0 AS k_min_value +FROM t_long_1_5_through_7_11 +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 200))) > 0 AS k_default_value +FROM t_long_1_5_through_7_11 +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 400))) > 0 AS k_custom_value +FROM t_long_1_5_through_7_11 +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col1, 65535))) > 0 AS k_max_value +FROM t_long_1_5_through_7_11 +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_agg_float(col1, 100))) > 0 AS k_float_sketch +FROM t_float_1_5_through_7_11 +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_agg_double(col1, 300))) > 0 AS k_double_sketch +FROM t_double_1_5_through_7_11 +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_bigint +FROM t_long_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_byte +FROM t_byte_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_short +FROM t_short_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1)) AS n_int +FROM t_int_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT kll_sketch_get_n_float(kll_sketch_agg_float(col1)) AS n_float +FROM t_float_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT kll_sketch_get_n_double(kll_sketch_agg_double(col1)) AS n_double +FROM t_double_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1, 100)) AS n_k_100 +FROM t_long_1_5_through_7_11 +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "(\"TINYINT\" or \"INT\" or \"BIGINT\" or \"SMALLINT\")", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 273, + "stopIndex" : 299, + "fragment" : "kll_sketch_agg_bigint(col1)" + } ] +} + + +-- !query +SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "(\"TINYINT\" or \"INT\" or \"BIGINT\" or \"SMALLINT\")", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 273, + "stopIndex" : 299, + "fragment" : "kll_sketch_agg_bigint(col1)" + } ] +} + + +-- !query +SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, + abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 274, + "stopIndex" : 299, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_bigint +FROM t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_int +FROM t_int_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_short +FROM t_short_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1) AS invalid_float_byte +FROM t_byte_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"FLOAT\"", + "sqlExpr" : "\"kll_sketch_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_sketch_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_bigint +FROM t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_int +FROM t_int_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_short +FROM t_short_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1) AS invalid_double_byte +FROM t_byte_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "(\"FLOAT\" or \"DOUBLE\")", + "sqlExpr" : "\"kll_sketch_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, -0.5) AS invalid_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_QUANTILE_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "functionName" : "`kll_sketch_get_quantile_bigint`", + "quantile" : "-0.5D" + } +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, 1.5) AS invalid_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_QUANTILE_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "functionName" : "`kll_sketch_get_quantile_bigint`", + "quantile" : "1.5D" + } +} + + +-- !query +SELECT kll_sketch_get_quantile_float(agg, array(-0.1, 0.5, 1.5)) AS invalid_quantiles +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_QUANTILE_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "functionName" : "`kll_sketch_get_quantile_float`", + "quantile" : "-0.1D" + } +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, 5) AS wrong_type +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "sqlState" : "22000", + "messageParameters" : { + "functionName" : "`kll_sketch_get_rank_bigint`", + "reason" : "reqOffset: 40, reqLength: 56, (reqOff + reqLen): 96, allocSize: 60" + } +} + + +-- !query +SELECT kll_sketch_merge_bigint(agg1, agg2) AS incompatible_merge +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg1, + kll_sketch_agg_float(CAST(col1 AS FLOAT)) AS agg2 + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INCOMPATIBLE_MERGE", + "sqlState" : "22000", + "messageParameters" : { + "functionName" : "`kll_sketch_merge_bigint`", + "reason" : "reqOffset: 40, reqLength: 56, (reqOff + reqLen): 96, allocSize: 60" + } +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(CAST('not_a_sketch' AS BINARY), 0.5) AS invalid_binary +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "sqlState" : "22000", + "messageParameters" : { + "functionName" : "`kll_sketch_get_quantile_bigint`", + "reason" : "Error combination of PreInts and SerVer: PreInts: 110, SerVer: 111" + } +} + + +-- !query +SELECT kll_sketch_get_quantile_float(agg, 0.5) IS NOT NULL AS returns_value +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT kll_sketch_to_string_double(agg) LIKE '%Kll%' AS contains_kll_header +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT kll_sketch_agg_bigint(col1, 7) AS k_too_small +FROM t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_sketch_agg_bigint`, the k parameter must be between 8 and 65535 (inclusive), but got 7. SQLSTATE: 22003", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1, 7)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "kll_sketch_agg_bigint(col1, 7)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_bigint(col1, 65536) AS k_too_large +FROM t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_sketch_agg_bigint`, the k parameter must be between 8 and 65535 (inclusive), but got 65536. SQLSTATE: 22003", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1, 65536)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "kll_sketch_agg_bigint(col1, 65536)" + } ] +} + + +-- !query +SELECT kll_sketch_agg_float(col1, CAST(NULL AS INT)) AS k_is_null +FROM t_float_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "k", + "sqlExpr" : "\"kll_sketch_agg_float(col1, CAST(NULL AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "kll_sketch_agg_float(col1, CAST(NULL AS INT))" + } ] +} + + +-- !query +SELECT kll_sketch_agg_double(col1, CAST(col1 AS INT)) AS k_non_constant +FROM t_double_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "CAST(spark_catalog.default.t_double_1_5_through_7_11.col1 AS INT)", + "inputName" : "k", + "inputType" : "int", + "sqlExpr" : "\"kll_sketch_agg_double(col1, CAST(col1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "kll_sketch_agg_double(col1, CAST(col1 AS INT))" + } ] +} + + +-- !query +SELECT kll_sketch_agg_bigint(col1, '100') AS k_wrong_type +FROM t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"100\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"INT\"", + "sqlExpr" : "\"kll_sketch_agg_bigint(col1, 100)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "kll_sketch_agg_bigint(col1, '100')" + } ] +} + + +-- !query +SELECT kll_sketch_get_n_bigint(X'deadbeef') AS invalid_binary_bigint +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "sqlState" : "22000", + "messageParameters" : { + "functionName" : "`kll_sketch_get_n_bigint`", + "reason" : "A sketch memory image must be at least 8 bytes. 4" + } +} + + +-- !query +SELECT kll_sketch_get_n_float(X'cafebabe') AS invalid_binary_float +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "sqlState" : "22000", + "messageParameters" : { + "functionName" : "`kll_sketch_get_n_float`", + "reason" : "A sketch memory image must be at least 8 bytes. 4" + } +} + + +-- !query +SELECT kll_sketch_get_n_double(X'12345678') AS invalid_binary_double +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "sqlState" : "22000", + "messageParameters" : { + "functionName" : "`kll_sketch_get_n_double`", + "reason" : "A sketch memory image must be at least 8 bytes. 4" + } +} + + +-- !query +DROP TABLE IF EXISTS t_int_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_short_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_byte_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_float_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t_double_1_5_through_7_11 +-- !query schema +struct<> +-- !query output + From aa01742b31ed646c3d2546ea682efd205b6a1bdf Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 6 Nov 2025 14:48:31 -0800 Subject: [PATCH 098/400] [SPARK-54199][SQL] Add DataFrame API support for new KLL quantiles sketch functions ### What changes were proposed in this pull request? This PR adds DataFrame API support for the KLL quantile sketch functions that were previously added to Spark SQL in https://github.com/apache/spark/pull/52800. This lets users leverage KLL sketches through both Scala and Python DataFrame APIs in addition to the existing SQL interface. **Key additions:** 1. **Scala DataFrame API** (`sql/api/src/main/scala/org/apache/spark/sql/functions.scala`): - 18 new functions covering aggregate, merge, quantile, and rank operations - Multiple overloads for each function supporting: - `Column` parameters for computed values - `String` parameters for column names - `Int` parameters for literal k values - Optional k parameters with sensible defaults - Functions for all three data type variants: bigint, float, double 2. **Python DataFrame API** (`python/pyspark/sql/functions/builtin.py`): - 18 corresponding Python functions with: - Comprehensive docstrings with usage examples - Proper type hints (`ColumnOrName`, `Optional[Union[int, Column]]`) - Support for both column objects and column name strings - Added to PySpark documentation reference 3. **Python Spark Connect Support** (`python/pyspark/sql/connect/functions/builtin.py`): - Full compatibility with Spark Connect architecture - All 18 functions properly registered ### Why are the changes needed? While the SQL API for KLL sketches was previously added, DataFrame API support is essential for full usability. Without DataFrame API support, users would be forced to use SQL expressions via `expr()` or `selectExpr()`, which is less ergonomic and type-safe. ### Does this PR introduce any user-facing change? Yes, this PR adds DataFrame API support for the 18 KLL sketch functions: **Scala DataFrame API Example:** ```scala import org.apache.spark.sql.functions._ // Create sketch with default k val df = Seq(1, 2, 3, 4, 5).toDF("value") val sketch = df.agg(kll_sketch_agg_bigint($"value")) // Create sketch with custom k value val sketch2 = df.agg(kll_sketch_agg_bigint("value", 400)) // Get median (0.5 quantile) val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) val median = sketchDf.select(kll_sketch_get_quantile_bigint($"sketch", lit(0.5))) // Get multiple quantiles val quantiles = sketchDf.select( kll_sketch_get_quantile_bigint($"sketch", array(lit(0.25), lit(0.5), lit(0.75))) ) // Merge sketches val merged = sketchDf.select( kll_sketch_merge_bigint($"sketch", $"sketch").alias("merged") ) // Get count of items val count = sketchDf.select(kll_sketch_get_n_bigint($"sketch")) ``` **Python DataFrame API Example:** ```python from pyspark.sql import functions as sf # Create sketch with default k df = spark.createDataFrame([1, 2, 3, 4, 5], "INT") sketch = df.agg(sf.kll_sketch_agg_bigint("value")) # Create sketch with custom k value sketch2 = df.agg(sf.kll_sketch_agg_bigint("value", 400)) # Get median (0.5 quantile) sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) median = sketch_df.select(sf.kll_sketch_get_quantile_bigint("sketch", sf.lit(0.5))) # Get multiple quantiles quantiles = sketch_df.select( sf.kll_sketch_get_quantile_bigint("sketch", sf.array(sf.lit(0.25), sf.lit(0.5), sf.lit(0.75))) ) # Merge sketches merged = sketch_df.select( sf.kll_sketch_merge_bigint("sketch", "sketch").alias("merged") ) # Get count of items count = sketch_df.select(sf.kll_sketch_get_n_bigint("sketch")) ``` ### How was this patch tested? 1. **Scala Unit Tests** (`DataFrameAggregateSuite`): - `kll_sketch_agg_{bigint,float,double}` with default and explicit k values - `kll_sketch_to_string` functions for all data types - `kll_sketch_get_n` functions for all data types - `kll_sketch_merge` operations - `kll_sketch_get_quantile` with single rank and array of ranks - `kll_sketch_get_rank` operations - Null value handling tests 2. **Python Unit Tests** (`test_functions.py`): - Comprehensive tests mirroring Scala tests - Tests for Column object and string column name overloads - Tests for optional k parameter - Array input tests for quantile/rank functions - Null handling validation - Type checking (bytes/bytearray for sketches, str for to_string, int/float for values) ### Was this patch authored or co-authored using generative AI tooling? Yes, IDE assistance used `claude-4.5-sonnet` with manual validation and integration. Closes #52900 from dtenedor/dataframe-api-kll-functions. Authored-by: Daniel Tenedorio Signed-off-by: Daniel Tenedorio --- .../reference/pyspark.sql/functions.rst | 18 + .../pyspark/sql/connect/functions/builtin.py | 162 +++++ python/pyspark/sql/functions/__init__.py | 18 + python/pyspark/sql/functions/builtin.py | 618 ++++++++++++++++++ python/pyspark/sql/tests/test_functions.py | 156 +++++ .../org/apache/spark/sql/functions.scala | 300 +++++++++ .../analyzer-results/kllquantiles.sql.out | 212 ++++++ .../sql-tests/inputs/kllquantiles.sql | 51 ++ .../sql-tests/results/kllquantiles.sql.out | 256 ++++++++ .../spark/sql/DataFrameAggregateSuite.scala | 164 +++++ 10 files changed, 1955 insertions(+) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index edd87c26dbb6b..280465af3e912 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -458,6 +458,9 @@ Aggregate Functions histogram_numeric hll_sketch_agg hll_union_agg + kll_sketch_agg_bigint + kll_sketch_agg_double + kll_sketch_agg_float kurtosis last last_value @@ -631,6 +634,21 @@ Misc Functions current_user hll_sketch_estimate hll_union + kll_sketch_get_n_bigint + kll_sketch_get_n_double + kll_sketch_get_n_float + kll_sketch_get_quantile_bigint + kll_sketch_get_quantile_double + kll_sketch_get_quantile_float + kll_sketch_get_rank_bigint + kll_sketch_get_rank_double + kll_sketch_get_rank_float + kll_sketch_merge_bigint + kll_sketch_merge_double + kll_sketch_merge_float + kll_sketch_to_string_bigint + kll_sketch_to_string_double + kll_sketch_to_string_float input_file_block_length input_file_block_start input_file_name diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 68caeef6ace83..2bd5b45ed1546 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -4529,6 +4529,168 @@ def theta_intersection_agg( theta_intersection_agg.__doc__ = pysparkfuncs.theta_intersection_agg.__doc__ +def kll_sketch_agg_bigint( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + fn = "kll_sketch_agg_bigint" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +kll_sketch_agg_bigint.__doc__ = pysparkfuncs.kll_sketch_agg_bigint.__doc__ + + +def kll_sketch_agg_float( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + fn = "kll_sketch_agg_float" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +kll_sketch_agg_float.__doc__ = pysparkfuncs.kll_sketch_agg_float.__doc__ + + +def kll_sketch_agg_double( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + fn = "kll_sketch_agg_double" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +kll_sketch_agg_double.__doc__ = pysparkfuncs.kll_sketch_agg_double.__doc__ + + +def kll_sketch_to_string_bigint(col: "ColumnOrName") -> Column: + fn = "kll_sketch_to_string_bigint" + return _invoke_function_over_columns(fn, col) + + +kll_sketch_to_string_bigint.__doc__ = pysparkfuncs.kll_sketch_to_string_bigint.__doc__ + + +def kll_sketch_to_string_float(col: "ColumnOrName") -> Column: + fn = "kll_sketch_to_string_float" + return _invoke_function_over_columns(fn, col) + + +kll_sketch_to_string_float.__doc__ = pysparkfuncs.kll_sketch_to_string_float.__doc__ + + +def kll_sketch_to_string_double(col: "ColumnOrName") -> Column: + fn = "kll_sketch_to_string_double" + return _invoke_function_over_columns(fn, col) + + +kll_sketch_to_string_double.__doc__ = pysparkfuncs.kll_sketch_to_string_double.__doc__ + + +def kll_sketch_get_n_bigint(col: "ColumnOrName") -> Column: + fn = "kll_sketch_get_n_bigint" + return _invoke_function_over_columns(fn, col) + + +kll_sketch_get_n_bigint.__doc__ = pysparkfuncs.kll_sketch_get_n_bigint.__doc__ + + +def kll_sketch_get_n_float(col: "ColumnOrName") -> Column: + fn = "kll_sketch_get_n_float" + return _invoke_function_over_columns(fn, col) + + +kll_sketch_get_n_float.__doc__ = pysparkfuncs.kll_sketch_get_n_float.__doc__ + + +def kll_sketch_get_n_double(col: "ColumnOrName") -> Column: + fn = "kll_sketch_get_n_double" + return _invoke_function_over_columns(fn, col) + + +kll_sketch_get_n_double.__doc__ = pysparkfuncs.kll_sketch_get_n_double.__doc__ + + +def kll_sketch_merge_bigint(left: "ColumnOrName", right: "ColumnOrName") -> Column: + fn = "kll_sketch_merge_bigint" + return _invoke_function_over_columns(fn, left, right) + + +kll_sketch_merge_bigint.__doc__ = pysparkfuncs.kll_sketch_merge_bigint.__doc__ + + +def kll_sketch_merge_float(left: "ColumnOrName", right: "ColumnOrName") -> Column: + fn = "kll_sketch_merge_float" + return _invoke_function_over_columns(fn, left, right) + + +kll_sketch_merge_float.__doc__ = pysparkfuncs.kll_sketch_merge_float.__doc__ + + +def kll_sketch_merge_double(left: "ColumnOrName", right: "ColumnOrName") -> Column: + fn = "kll_sketch_merge_double" + return _invoke_function_over_columns(fn, left, right) + + +kll_sketch_merge_double.__doc__ = pysparkfuncs.kll_sketch_merge_double.__doc__ + + +def kll_sketch_get_quantile_bigint(sketch: "ColumnOrName", rank: "ColumnOrName") -> Column: + fn = "kll_sketch_get_quantile_bigint" + return _invoke_function_over_columns(fn, sketch, rank) + + +kll_sketch_get_quantile_bigint.__doc__ = pysparkfuncs.kll_sketch_get_quantile_bigint.__doc__ + + +def kll_sketch_get_quantile_float(sketch: "ColumnOrName", rank: "ColumnOrName") -> Column: + fn = "kll_sketch_get_quantile_float" + return _invoke_function_over_columns(fn, sketch, rank) + + +kll_sketch_get_quantile_float.__doc__ = pysparkfuncs.kll_sketch_get_quantile_float.__doc__ + + +def kll_sketch_get_quantile_double(sketch: "ColumnOrName", rank: "ColumnOrName") -> Column: + fn = "kll_sketch_get_quantile_double" + return _invoke_function_over_columns(fn, sketch, rank) + + +kll_sketch_get_quantile_double.__doc__ = pysparkfuncs.kll_sketch_get_quantile_double.__doc__ + + +def kll_sketch_get_rank_bigint(sketch: "ColumnOrName", quantile: "ColumnOrName") -> Column: + fn = "kll_sketch_get_rank_bigint" + return _invoke_function_over_columns(fn, sketch, quantile) + + +kll_sketch_get_rank_bigint.__doc__ = pysparkfuncs.kll_sketch_get_rank_bigint.__doc__ + + +def kll_sketch_get_rank_float(sketch: "ColumnOrName", quantile: "ColumnOrName") -> Column: + fn = "kll_sketch_get_rank_float" + return _invoke_function_over_columns(fn, sketch, quantile) + + +kll_sketch_get_rank_float.__doc__ = pysparkfuncs.kll_sketch_get_rank_float.__doc__ + + +def kll_sketch_get_rank_double(sketch: "ColumnOrName", quantile: "ColumnOrName") -> Column: + fn = "kll_sketch_get_rank_double" + return _invoke_function_over_columns(fn, sketch, quantile) + + +kll_sketch_get_rank_double.__doc__ = pysparkfuncs.kll_sketch_get_rank_double.__doc__ + + def theta_sketch_estimate(col: "ColumnOrName") -> Column: fn = "theta_sketch_estimate" return _invoke_function_over_columns(fn, col) diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index fa579a222efad..173e074564534 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -372,6 +372,9 @@ "histogram_numeric", "hll_sketch_agg", "hll_union_agg", + "kll_sketch_agg_bigint", + "kll_sketch_agg_double", + "kll_sketch_agg_float", "kurtosis", "last", "last_value", @@ -495,6 +498,21 @@ "input_file_block_start", "input_file_name", "java_method", + "kll_sketch_get_n_bigint", + "kll_sketch_get_n_double", + "kll_sketch_get_n_float", + "kll_sketch_get_quantile_bigint", + "kll_sketch_get_quantile_double", + "kll_sketch_get_quantile_float", + "kll_sketch_get_rank_bigint", + "kll_sketch_get_rank_double", + "kll_sketch_get_rank_float", + "kll_sketch_merge_bigint", + "kll_sketch_merge_double", + "kll_sketch_merge_float", + "kll_sketch_to_string_bigint", + "kll_sketch_to_string_double", + "kll_sketch_to_string_float", "monotonically_increasing_id", "raise_error", "reflect", diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 26c498dfa252a..f0fbe1529fea2 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -26536,6 +26536,624 @@ def theta_intersection_agg(col: "ColumnOrName") -> Column: return _invoke_function_over_columns(fn, col) +@_try_remote_functions +def kll_sketch_agg_bigint( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + """ + Aggregate function: returns the compact binary representation of the Datasketches + KllLongsSketch built with the values in the input column. The optional k parameter + controls the size and accuracy of the sketch (default 200, range 8-65535). + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The column containing bigint values to aggregate + k : :class:`~pyspark.sql.Column` or int, optional + The k parameter that controls size and accuracy (default 200, range 8-65535) + + Returns + ------- + :class:`~pyspark.sql.Column` + The binary representation of the KllLongsSketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1,2,3,4,5], "INT") + >>> result = df.agg(sf.kll_sketch_agg_bigint("value")).first()[0] + >>> result is not None and len(result) > 0 + True + """ + fn = "kll_sketch_agg_bigint" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +@_try_remote_functions +def kll_sketch_agg_float( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + """ + Aggregate function: returns the compact binary representation of the Datasketches + KllFloatsSketch built with the values in the input column. The optional k parameter + controls the size and accuracy of the sketch (default 200, range 8-65535). + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The column containing float values to aggregate + k : :class:`~pyspark.sql.Column` or int, optional + The k parameter that controls size and accuracy (default 200, range 8-65535) + + Returns + ------- + :class:`~pyspark.sql.Column` + The binary representation of the KllFloatsSketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") + >>> result = df.agg(sf.kll_sketch_agg_float("value")).first()[0] + >>> result is not None and len(result) > 0 + True + """ + fn = "kll_sketch_agg_float" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +@_try_remote_functions +def kll_sketch_agg_double( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + """ + Aggregate function: returns the compact binary representation of the Datasketches + KllDoublesSketch built with the values in the input column. The optional k parameter + controls the size and accuracy of the sketch (default 200, range 8-65535). + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The column containing double values to aggregate + k : :class:`~pyspark.sql.Column` or int, optional + The k parameter that controls size and accuracy (default 200, range 8-65535) + + Returns + ------- + :class:`~pyspark.sql.Column` + The binary representation of the KllDoublesSketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") + >>> result = df.agg(sf.kll_sketch_agg_double("value")).first()[0] + >>> result is not None and len(result) > 0 + True + """ + fn = "kll_sketch_agg_double" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +@_try_remote_functions +def kll_sketch_to_string_bigint(col: "ColumnOrName") -> Column: + """ + Returns a string with human readable summary information about the KLL bigint sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The KLL bigint sketch binary representation + + Returns + ------- + :class:`~pyspark.sql.Column` + A string representation of the sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1,2,3,4,5], "INT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> result = sketch_df.select(sf.kll_sketch_to_string_bigint("sketch")).first()[0] + >>> "Kll" in result and "N" in result + True + """ + fn = "kll_sketch_to_string_bigint" + return _invoke_function_over_columns(fn, col) + + +@_try_remote_functions +def kll_sketch_to_string_float(col: "ColumnOrName") -> Column: + """ + Returns a string with human readable summary information about the KLL float sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The KLL float sketch binary representation + + Returns + ------- + :class:`~pyspark.sql.Column` + A string representation of the sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> result = sketch_df.select(sf.kll_sketch_to_string_float("sketch")).first()[0] + >>> "Kll" in result and "N" in result + True + """ + fn = "kll_sketch_to_string_float" + return _invoke_function_over_columns(fn, col) + + +@_try_remote_functions +def kll_sketch_to_string_double(col: "ColumnOrName") -> Column: + """ + Returns a string with human readable summary information about the KLL double sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The KLL double sketch binary representation + + Returns + ------- + :class:`~pyspark.sql.Column` + A string representation of the sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") + >>> sketch_df = df.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> result = sketch_df.select(sf.kll_sketch_to_string_double("sketch")).first()[0] + >>> "Kll" in result and "N" in result + True + """ + fn = "kll_sketch_to_string_double" + return _invoke_function_over_columns(fn, col) + + +@_try_remote_functions +def kll_sketch_get_n_bigint(col: "ColumnOrName") -> Column: + """ + Returns the number of items collected in the KLL bigint sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The KLL bigint sketch binary representation + + Returns + ------- + :class:`~pyspark.sql.Column` + The count of items in the sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1,2,3,4,5], "INT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_n_bigint("sketch")).show() + +-------------------------------+ + |kll_sketch_get_n_bigint(sketch)| + +-------------------------------+ + | 5| + +-------------------------------+ + """ + fn = "kll_sketch_get_n_bigint" + return _invoke_function_over_columns(fn, col) + + +@_try_remote_functions +def kll_sketch_get_n_float(col: "ColumnOrName") -> Column: + """ + Returns the number of items collected in the KLL float sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The KLL float sketch binary representation + + Returns + ------- + :class:`~pyspark.sql.Column` + The count of items in the sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_n_float("sketch")).show() + +------------------------------+ + |kll_sketch_get_n_float(sketch)| + +------------------------------+ + | 5| + +------------------------------+ + """ + fn = "kll_sketch_get_n_float" + return _invoke_function_over_columns(fn, col) + + +@_try_remote_functions +def kll_sketch_get_n_double(col: "ColumnOrName") -> Column: + """ + Returns the number of items collected in the KLL double sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The KLL double sketch binary representation + + Returns + ------- + :class:`~pyspark.sql.Column` + The count of items in the sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") + >>> sketch_df = df.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_n_double("sketch")).show() + +-------------------------------+ + |kll_sketch_get_n_double(sketch)| + +-------------------------------+ + | 5| + +-------------------------------+ + """ + fn = "kll_sketch_get_n_double" + return _invoke_function_over_columns(fn, col) + + +@_try_remote_functions +def kll_sketch_merge_bigint(left: "ColumnOrName", right: "ColumnOrName") -> Column: + """ + Merges two KLL bigint sketch buffers together into one. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + left : :class:`~pyspark.sql.Column` or column name + The first KLL bigint sketch + right : :class:`~pyspark.sql.Column` or column name + The second KLL bigint sketch + + Returns + ------- + :class:`~pyspark.sql.Column` + The merged KLL sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1,2,3,4,5], "INT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> result = sketch_df.select(sf.kll_sketch_merge_bigint("sketch", "sketch")).first()[0] + >>> result is not None and len(result) > 0 + True + """ + fn = "kll_sketch_merge_bigint" + return _invoke_function_over_columns(fn, left, right) + + +@_try_remote_functions +def kll_sketch_merge_float(left: "ColumnOrName", right: "ColumnOrName") -> Column: + """ + Merges two KLL float sketch buffers together into one. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + left : :class:`~pyspark.sql.Column` or column name + The first KLL float sketch + right : :class:`~pyspark.sql.Column` or column name + The second KLL float sketch + + Returns + ------- + :class:`~pyspark.sql.Column` + The merged KLL sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> result = sketch_df.select(sf.kll_sketch_merge_float("sketch", "sketch")).first()[0] + >>> result is not None and len(result) > 0 + True + """ + fn = "kll_sketch_merge_float" + return _invoke_function_over_columns(fn, left, right) + + +@_try_remote_functions +def kll_sketch_merge_double(left: "ColumnOrName", right: "ColumnOrName") -> Column: + """ + Merges two KLL double sketch buffers together into one. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + left : :class:`~pyspark.sql.Column` or column name + The first KLL double sketch + right : :class:`~pyspark.sql.Column` or column name + The second KLL double sketch + + Returns + ------- + :class:`~pyspark.sql.Column` + The merged KLL sketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") + >>> sketch_df = df.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> result = sketch_df.select(sf.kll_sketch_merge_double("sketch", "sketch")).first()[0] + >>> result is not None and len(result) > 0 + True + """ + fn = "kll_sketch_merge_double" + return _invoke_function_over_columns(fn, left, right) + + +@_try_remote_functions +def kll_sketch_get_quantile_bigint(sketch: "ColumnOrName", rank: "ColumnOrName") -> Column: + """ + Extracts a quantile value from a KLL bigint sketch given an input rank value. + The rank can be a single value or an array. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + sketch : :class:`~pyspark.sql.Column` or column name + The KLL bigint sketch binary representation + rank : :class:`~pyspark.sql.Column` or column name + The rank value(s) to extract (between 0.0 and 1.0) + + Returns + ------- + :class:`~pyspark.sql.Column` + The quantile value(s). + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1,2,3,4,5], "INT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_quantile_bigint("sketch", sf.lit(0.5))).show() + +-------------------------------------------+ + |kll_sketch_get_quantile_bigint(sketch, 0.5)| + +-------------------------------------------+ + | 3| + +-------------------------------------------+ + """ + fn = "kll_sketch_get_quantile_bigint" + return _invoke_function_over_columns(fn, sketch, rank) + + +@_try_remote_functions +def kll_sketch_get_quantile_float(sketch: "ColumnOrName", rank: "ColumnOrName") -> Column: + """ + Extracts a quantile value from a KLL float sketch given an input rank value. + The rank can be a single value or an array. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + sketch : :class:`~pyspark.sql.Column` or column name + The KLL float sketch binary representation + rank : :class:`~pyspark.sql.Column` or column name + The rank value(s) to extract (between 0.0 and 1.0) + + Returns + ------- + :class:`~pyspark.sql.Column` + The quantile value(s). + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_quantile_float("sketch", sf.lit(0.5))).show() + +------------------------------------------+ + |kll_sketch_get_quantile_float(sketch, 0.5)| + +------------------------------------------+ + | 3.0| + +------------------------------------------+ + """ + fn = "kll_sketch_get_quantile_float" + return _invoke_function_over_columns(fn, sketch, rank) + + +@_try_remote_functions +def kll_sketch_get_quantile_double(sketch: "ColumnOrName", rank: "ColumnOrName") -> Column: + """ + Extracts a quantile value from a KLL double sketch given an input rank value. + The rank can be a single value or an array. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + sketch : :class:`~pyspark.sql.Column` or column name + The KLL double sketch binary representation + rank : :class:`~pyspark.sql.Column` or column name + The rank value(s) to extract (between 0.0 and 1.0) + + Returns + ------- + :class:`~pyspark.sql.Column` + The quantile value(s). + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") + >>> sketch_df = df.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_quantile_double("sketch", sf.lit(0.5))).show() + +-------------------------------------------+ + |kll_sketch_get_quantile_double(sketch, 0.5)| + +-------------------------------------------+ + | 3.0| + +-------------------------------------------+ + """ + fn = "kll_sketch_get_quantile_double" + return _invoke_function_over_columns(fn, sketch, rank) + + +@_try_remote_functions +def kll_sketch_get_rank_bigint(sketch: "ColumnOrName", quantile: "ColumnOrName") -> Column: + """ + Extracts a rank value from a KLL bigint sketch given an input quantile value. + The quantile can be a single value or an array. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + sketch : :class:`~pyspark.sql.Column` or column name + The KLL bigint sketch binary representation + quantile : :class:`~pyspark.sql.Column` or column name + The quantile value(s) to lookup + + Returns + ------- + :class:`~pyspark.sql.Column` + The rank value(s) (between 0.0 and 1.0). + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1,2,3,4,5], "INT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_rank_bigint("sketch", sf.lit(3))).show() + +-------------------------------------+ + |kll_sketch_get_rank_bigint(sketch, 3)| + +-------------------------------------+ + | 0.6| + +-------------------------------------+ + """ + fn = "kll_sketch_get_rank_bigint" + return _invoke_function_over_columns(fn, sketch, quantile) + + +@_try_remote_functions +def kll_sketch_get_rank_float(sketch: "ColumnOrName", quantile: "ColumnOrName") -> Column: + """ + Extracts a rank value from a KLL float sketch given an input quantile value. + The quantile can be a single value or an array. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + sketch : :class:`~pyspark.sql.Column` or column name + The KLL float sketch binary representation + quantile : :class:`~pyspark.sql.Column` or column name + The quantile value(s) to lookup + + Returns + ------- + :class:`~pyspark.sql.Column` + The rank value(s) (between 0.0 and 1.0). + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") + >>> sketch_df = df.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_rank_float("sketch", sf.lit(3.0))).show() + +--------------------------------------+ + |kll_sketch_get_rank_float(sketch, 3.0)| + +--------------------------------------+ + | 0.6| + +--------------------------------------+ + """ + fn = "kll_sketch_get_rank_float" + return _invoke_function_over_columns(fn, sketch, quantile) + + +@_try_remote_functions +def kll_sketch_get_rank_double(sketch: "ColumnOrName", quantile: "ColumnOrName") -> Column: + """ + Extracts a rank value from a KLL double sketch given an input quantile value. + The quantile can be a single value or an array. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + sketch : :class:`~pyspark.sql.Column` or column name + The KLL double sketch binary representation + quantile : :class:`~pyspark.sql.Column` or column name + The quantile value(s) to lookup + + Returns + ------- + :class:`~pyspark.sql.Column` + The rank value(s) (between 0.0 and 1.0). + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") + >>> sketch_df = df.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> sketch_df.select(sf.kll_sketch_get_rank_double("sketch", sf.lit(3.0))).show() + +---------------------------------------+ + |kll_sketch_get_rank_double(sketch, 3.0)| + +---------------------------------------+ + | 0.6| + +---------------------------------------+ + """ + fn = "kll_sketch_get_rank_double" + return _invoke_function_over_columns(fn, sketch, quantile) + + @_try_remote_functions def theta_sketch_estimate(col: "ColumnOrName") -> Column: """ diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 0f646b3da549a..a1a4f19e05227 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -2114,6 +2114,162 @@ def test_listagg_distinct_functions(self): None, ) + def test_kll_sketch_agg_bigint(self): + """Test kll_sketch_agg_bigint function""" + df = self.spark.createDataFrame([1, 2, 3, 4, 5], "INT") + + # Test with default k + sketch = df.agg(F.kll_sketch_agg_bigint("value")).first()[0] + self.assertIsNotNone(sketch) + self.assertIsInstance(sketch, (bytes, bytearray)) + + # Test with explicit k + sketch_k = df.agg(F.kll_sketch_agg_bigint("value", 400)).first()[0] + self.assertIsNotNone(sketch_k) + + def test_kll_sketch_agg_float(self): + """Test kll_sketch_agg_float function""" + df = self.spark.createDataFrame([1.0, 2.0, 3.0, 4.0, 5.0], "FLOAT") + + sketch = df.agg(F.kll_sketch_agg_float("value")).first()[0] + self.assertIsNotNone(sketch) + self.assertIsInstance(sketch, (bytes, bytearray)) + + def test_kll_sketch_agg_double(self): + """Test kll_sketch_agg_double function""" + df = self.spark.createDataFrame([1.0, 2.0, 3.0, 4.0, 5.0], "DOUBLE") + + sketch = df.agg(F.kll_sketch_agg_double("value")).first()[0] + self.assertIsNotNone(sketch) + self.assertIsInstance(sketch, (bytes, bytearray)) + + def test_kll_sketch_to_string_bigint(self): + """Test kll_sketch_to_string_bigint function""" + df = self.spark.createDataFrame([1, 2, 3, 4, 5], "INT") + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + result = sketch_df.select(F.kll_sketch_to_string_bigint("sketch")).first()[0] + self.assertIsNotNone(result) + self.assertIsInstance(result, str) + self.assertIn("Kll", result) + + def test_kll_sketch_get_n_bigint(self): + """Test kll_sketch_get_n_bigint function""" + df = self.spark.createDataFrame([1, 2, 3, 4, 5], "INT") + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + n = sketch_df.select(F.kll_sketch_get_n_bigint("sketch")).first()[0] + self.assertEqual(n, 5) + + def test_kll_sketch_merge_bigint(self): + """Test kll_sketch_merge_bigint function""" + df = self.spark.createDataFrame([1, 2, 3], "INT") + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + merged = sketch_df.select(F.kll_sketch_merge_bigint("sketch", "sketch")).first()[0] + self.assertIsNotNone(merged) + self.assertIsInstance(merged, (bytes, bytearray)) + + def test_kll_sketch_get_quantile_bigint(self): + """Test kll_sketch_get_quantile_bigint function""" + df = self.spark.createDataFrame([1, 2, 3, 4, 5], "INT") + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + quantile = sketch_df.select(F.kll_sketch_get_quantile_bigint("sketch", F.lit(0.5))).first()[ + 0 + ] + self.assertIsNotNone(quantile) + self.assertGreaterEqual(quantile, 1) + self.assertLessEqual(quantile, 5) + + def test_kll_sketch_get_quantile_bigint_array(self): + """Test kll_sketch_get_quantile_bigint with array of ranks""" + df = self.spark.createDataFrame([1, 2, 3, 4, 5], "INT") + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + quantiles = sketch_df.select( + F.kll_sketch_get_quantile_bigint( + "sketch", F.array(F.lit(0.25), F.lit(0.5), F.lit(0.75)) + ) + ).first()[0] + self.assertIsNotNone(quantiles) + self.assertEqual(len(quantiles), 3) + + def test_kll_sketch_get_rank_bigint(self): + """Test kll_sketch_get_rank_bigint function""" + df = self.spark.createDataFrame([1, 2, 3, 4, 5], "INT") + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + rank = sketch_df.select(F.kll_sketch_get_rank_bigint("sketch", F.lit(3))).first()[0] + self.assertIsNotNone(rank) + self.assertGreaterEqual(rank, 0.0) + self.assertLessEqual(rank, 1.0) + + def test_kll_sketch_float_variants(self): + """Test all float variant functions""" + df = self.spark.createDataFrame([1.0, 2.0, 3.0, 4.0, 5.0], "FLOAT") + sketch_df = df.agg(F.kll_sketch_agg_float("value").alias("sketch")) + + # Test to_string + string_result = sketch_df.select(F.kll_sketch_to_string_float("sketch")).first()[0] + self.assertIn("Kll", string_result) + + # Test get_n + n = sketch_df.select(F.kll_sketch_get_n_float("sketch")).first()[0] + self.assertEqual(n, 5) + + # Test merge + merged = sketch_df.select(F.kll_sketch_merge_float("sketch", "sketch")).first()[0] + self.assertIsNotNone(merged) + + # Test get_quantile + quantile = sketch_df.select(F.kll_sketch_get_quantile_float("sketch", F.lit(0.5))).first()[ + 0 + ] + self.assertIsNotNone(quantile) + + # Test get_rank + rank = sketch_df.select(F.kll_sketch_get_rank_float("sketch", F.lit(3.0))).first()[0] + self.assertGreaterEqual(rank, 0.0) + self.assertLessEqual(rank, 1.0) + + def test_kll_sketch_double_variants(self): + """Test all double variant functions""" + df = self.spark.createDataFrame([1.0, 2.0, 3.0, 4.0, 5.0], "DOUBLE") + sketch_df = df.agg(F.kll_sketch_agg_double("value").alias("sketch")) + + # Test to_string + string_result = sketch_df.select(F.kll_sketch_to_string_double("sketch")).first()[0] + self.assertIn("Kll", string_result) + + # Test get_n + n = sketch_df.select(F.kll_sketch_get_n_double("sketch")).first()[0] + self.assertEqual(n, 5) + + # Test merge + merged = sketch_df.select(F.kll_sketch_merge_double("sketch", "sketch")).first()[0] + self.assertIsNotNone(merged) + + # Test get_quantile + quantile = sketch_df.select(F.kll_sketch_get_quantile_double("sketch", F.lit(0.5))).first()[ + 0 + ] + self.assertIsNotNone(quantile) + + # Test get_rank + rank = sketch_df.select(F.kll_sketch_get_rank_double("sketch", F.lit(3.0))).first()[0] + self.assertGreaterEqual(rank, 0.0) + self.assertLessEqual(rank, 1.0) + + def test_kll_sketch_with_nulls(self): + """Test KLL sketch with null values""" + df = self.spark.createDataFrame([(1,), (None,), (3,), (4,), (None,)], ["value"]) + sketch_df = df.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + n = sketch_df.select(F.kll_sketch_get_n_bigint("sketch")).first()[0] + # Should only count non-null values + self.assertEqual(n, 3) + def test_datetime_functions(self): df = self.spark.range(1).selectExpr("'2017-01-22' as dateCol") parse_result = df.select(F.to_date(F.col("dateCol"))).first() diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index ab883d5933cf5..6f56c86541e43 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -1303,6 +1303,306 @@ object functions { def theta_union_agg(columnName: String): Column = theta_union_agg(Column(columnName)) + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllLongsSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_bigint(e: Column, k: Column): Column = + Column.fn("kll_sketch_agg_bigint", e, k) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllLongsSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_bigint(e: Column, k: Int): Column = + Column.fn("kll_sketch_agg_bigint", e, lit(k)) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllLongsSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_bigint(columnName: String, k: Int): Column = + kll_sketch_agg_bigint(Column(columnName), k) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllLongsSketch built with the values in the input column with default k value of 200. + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_bigint(e: Column): Column = + Column.fn("kll_sketch_agg_bigint", e) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllLongsSketch built with the values in the input column with default k value of 200. + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_bigint(columnName: String): Column = + kll_sketch_agg_bigint(Column(columnName)) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllFloatsSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_float(e: Column, k: Column): Column = + Column.fn("kll_sketch_agg_float", e, k) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllFloatsSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_float(e: Column, k: Int): Column = + Column.fn("kll_sketch_agg_float", e, lit(k)) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllFloatsSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_float(columnName: String, k: Int): Column = + kll_sketch_agg_float(Column(columnName), k) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllFloatsSketch built with the values in the input column with default k value of 200. + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_float(e: Column): Column = + Column.fn("kll_sketch_agg_float", e) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllFloatsSketch built with the values in the input column with default k value of 200. + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_float(columnName: String): Column = + kll_sketch_agg_float(Column(columnName)) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllDoublesSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_double(e: Column, k: Column): Column = + Column.fn("kll_sketch_agg_double", e, k) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllDoublesSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_double(e: Column, k: Int): Column = + Column.fn("kll_sketch_agg_double", e, lit(k)) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllDoublesSketch built with the values in the input column. The optional k parameter controls + * the size and accuracy of the sketch (default 200, range 8-65535). + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_double(columnName: String, k: Int): Column = + kll_sketch_agg_double(Column(columnName), k) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllDoublesSketch built with the values in the input column with default k value of 200. + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_double(e: Column): Column = + Column.fn("kll_sketch_agg_double", e) + + /** + * Aggregate function: returns the compact binary representation of the Datasketches + * KllDoublesSketch built with the values in the input column with default k value of 200. + * + * @group agg_funcs + * @since 4.1.0 + */ + def kll_sketch_agg_double(columnName: String): Column = + kll_sketch_agg_double(Column(columnName)) + + /** + * Returns a string with human readable summary information about the KLL bigint sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_to_string_bigint(e: Column): Column = + Column.fn("kll_sketch_to_string_bigint", e) + + /** + * Returns a string with human readable summary information about the KLL float sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_to_string_float(e: Column): Column = + Column.fn("kll_sketch_to_string_float", e) + + /** + * Returns a string with human readable summary information about the KLL double sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_to_string_double(e: Column): Column = + Column.fn("kll_sketch_to_string_double", e) + + /** + * Returns the number of items collected in the KLL bigint sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_n_bigint(e: Column): Column = + Column.fn("kll_sketch_get_n_bigint", e) + + /** + * Returns the number of items collected in the KLL float sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_n_float(e: Column): Column = + Column.fn("kll_sketch_get_n_float", e) + + /** + * Returns the number of items collected in the KLL double sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_n_double(e: Column): Column = + Column.fn("kll_sketch_get_n_double", e) + + /** + * Merges two KLL bigint sketch buffers together into one. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_merge_bigint(left: Column, right: Column): Column = + Column.fn("kll_sketch_merge_bigint", left, right) + + /** + * Merges two KLL float sketch buffers together into one. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_merge_float(left: Column, right: Column): Column = + Column.fn("kll_sketch_merge_float", left, right) + + /** + * Merges two KLL double sketch buffers together into one. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_merge_double(left: Column, right: Column): Column = + Column.fn("kll_sketch_merge_double", left, right) + + /** + * Extracts a quantile value from a KLL bigint sketch given an input rank value. The rank can be + * a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_quantile_bigint(sketch: Column, rank: Column): Column = + Column.fn("kll_sketch_get_quantile_bigint", sketch, rank) + + /** + * Extracts a quantile value from a KLL float sketch given an input rank value. The rank can be + * a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_quantile_float(sketch: Column, rank: Column): Column = + Column.fn("kll_sketch_get_quantile_float", sketch, rank) + + /** + * Extracts a quantile value from a KLL double sketch given an input rank value. The rank can be + * a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_quantile_double(sketch: Column, rank: Column): Column = + Column.fn("kll_sketch_get_quantile_double", sketch, rank) + + /** + * Extracts a rank value from a KLL bigint sketch given an input quantile value. The quantile + * can be a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_rank_bigint(sketch: Column, quantile: Column): Column = + Column.fn("kll_sketch_get_rank_bigint", sketch, quantile) + + /** + * Extracts a rank value from a KLL float sketch given an input quantile value. The quantile can + * be a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_rank_float(sketch: Column, quantile: Column): Column = + Column.fn("kll_sketch_get_rank_float", sketch, quantile) + + /** + * Extracts a rank value from a KLL double sketch given an input quantile value. The quantile + * can be a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_rank_double(sketch: Column, quantile: Column): Column = + Column.fn("kll_sketch_get_rank_double", sketch, quantile) + /** * Aggregate function: returns the concatenation of non-null input values. * diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out index 049cd163f628c..64fc8998c9e42 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out @@ -1082,6 +1082,218 @@ Project [kll_sketch_get_n_double(0x12345678) AS invalid_binary_double#xL] +- OneRowRelation +-- !query +SELECT kll_sketch_get_n_bigint(42) AS wrong_argument_type +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"42\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_sketch_get_n_bigint(42)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_get_n_bigint(42)" + } ] +} + + +-- !query +SELECT kll_sketch_get_n_float(42.0) AS wrong_argument_type +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"42.0\"", + "inputType" : "\"DECIMAL(3,1)\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_sketch_get_n_float(42.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "kll_sketch_get_n_float(42.0)" + } ] +} + + +-- !query +SELECT kll_sketch_get_n_double(42.0D) AS wrong_argument_type +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"42.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_sketch_get_n_double(42.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "kll_sketch_get_n_double(42.0D)" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, 'invalid') AS quantile_string +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_get_quantile_bigint(agg#x, cast(invalid as double)) AS quantile_string#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_get_quantile_float(agg, X'deadbeef') AS quantile_binary +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'DEADBEEF'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "second", + "requiredType" : "(\"DOUBLE\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_quantile_float(agg, X'DEADBEEF')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "kll_sketch_get_quantile_float(agg, X'deadbeef')" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_double(agg, true) AS quantile_boolean +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "second", + "requiredType" : "(\"DOUBLE\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_quantile_double(agg, true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "kll_sketch_get_quantile_double(agg, true)" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, 'invalid') AS rank_string +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query analysis +Project [kll_sketch_get_rank_bigint(agg#x, cast(invalid as bigint)) AS rank_string#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] + +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + + +-- !query +SELECT kll_sketch_get_rank_float(agg, X'cafebabe') AS rank_binary +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'CAFEBABE'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "second", + "requiredType" : "(\"FLOAT\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_rank_float(agg, X'CAFEBABE')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "kll_sketch_get_rank_float(agg, X'cafebabe')" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_double(agg, false) AS rank_boolean +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"false\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "second", + "requiredType" : "(\"DOUBLE\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_rank_double(agg, false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "kll_sketch_get_rank_double(agg, false)" + } ] +} + + -- !query DROP TABLE IF EXISTS t_int_1_5_through_7_11 -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql index 404235b33e670..d0d7fb1f9c127 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql @@ -413,6 +413,57 @@ SELECT kll_sketch_get_n_float(X'cafebabe') AS invalid_binary_float; SELECT kll_sketch_get_n_double(X'12345678') AS invalid_binary_double; +-- Wrong argument types +SELECT kll_sketch_get_n_bigint(42) AS wrong_argument_type; + +SELECT kll_sketch_get_n_float(42.0) AS wrong_argument_type; + +SELECT kll_sketch_get_n_double(42.0D) AS wrong_argument_type; + +-- Negative tests for kll_sketch_get_quantile functions with invalid second argument types +-- Invalid type: STRING instead of DOUBLE for quantile parameter +SELECT kll_sketch_get_quantile_bigint(agg, 'invalid') AS quantile_string +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +-- Invalid type: BINARY instead of DOUBLE for quantile parameter +SELECT kll_sketch_get_quantile_float(agg, X'deadbeef') AS quantile_binary +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- Invalid type: BOOLEAN instead of DOUBLE for quantile parameter +SELECT kll_sketch_get_quantile_double(agg, true) AS quantile_boolean +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +); + +-- Negative tests for kll_sketch_get_rank functions with invalid second argument types +-- Invalid type: STRING instead of BIGINT for rank value parameter +SELECT kll_sketch_get_rank_bigint(agg, 'invalid') AS rank_string +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +); + +-- Invalid type: BINARY instead of FLOAT for rank value parameter +SELECT kll_sketch_get_rank_float(agg, X'cafebabe') AS rank_binary +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +); + +-- Invalid type: BOOLEAN instead of DOUBLE for rank value parameter +SELECT kll_sketch_get_rank_double(agg, false) AS rank_boolean +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +); + -- Clean up DROP TABLE IF EXISTS t_int_1_5_through_7_11; DROP TABLE IF EXISTS t_long_1_5_through_7_11; diff --git a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out index 383a8e66fce3c..3618c851939e0 100644 --- a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out @@ -1139,6 +1139,262 @@ org.apache.spark.SparkRuntimeException } +-- !query +SELECT kll_sketch_get_n_bigint(42) AS wrong_argument_type +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"42\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_sketch_get_n_bigint(42)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "kll_sketch_get_n_bigint(42)" + } ] +} + + +-- !query +SELECT kll_sketch_get_n_float(42.0) AS wrong_argument_type +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"42.0\"", + "inputType" : "\"DECIMAL(3,1)\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_sketch_get_n_float(42.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "kll_sketch_get_n_float(42.0)" + } ] +} + + +-- !query +SELECT kll_sketch_get_n_double(42.0D) AS wrong_argument_type +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"42.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_sketch_get_n_double(42.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "kll_sketch_get_n_double(42.0D)" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, 'invalid') AS quantile_string +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'invalid'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "kll_sketch_get_quantile_bigint(agg, 'invalid')" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_float(agg, X'deadbeef') AS quantile_binary +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'DEADBEEF'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "second", + "requiredType" : "(\"DOUBLE\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_quantile_float(agg, X'DEADBEEF')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "kll_sketch_get_quantile_float(agg, X'deadbeef')" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_double(agg, true) AS quantile_boolean +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "second", + "requiredType" : "(\"DOUBLE\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_quantile_double(agg, true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "kll_sketch_get_quantile_double(agg, true)" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, 'invalid') AS rank_string +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg + FROM t_long_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'invalid'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "kll_sketch_get_rank_bigint(agg, 'invalid')" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_float(agg, X'cafebabe') AS rank_binary +FROM ( + SELECT kll_sketch_agg_float(col1) AS agg + FROM t_float_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'CAFEBABE'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "second", + "requiredType" : "(\"FLOAT\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_rank_float(agg, X'CAFEBABE')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "kll_sketch_get_rank_float(agg, X'cafebabe')" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_double(agg, false) AS rank_boolean +FROM ( + SELECT kll_sketch_agg_double(col1) AS agg + FROM t_double_1_5_through_7_11 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"false\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "second", + "requiredType" : "(\"DOUBLE\" or \"ARRAY\")", + "sqlExpr" : "\"kll_sketch_get_rank_double(agg, false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "kll_sketch_get_rank_double(agg, false)" + } ] +} + + -- !query DROP TABLE IF EXISTS t_int_1_5_through_7_11 -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 37614145fe83f..da2fbceae97e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -3302,6 +3302,170 @@ class DataFrameAggregateSuite extends QueryTest .groupBy($"col1").agg(max("col1")) checkAnswer(df, Seq(Row(1, 1))) } + + test("kll_sketch_agg_bigint basic functionality") { + val df = Seq(1, 2, 3, 4, 5).toDF("value") + + // Test with default k + val sketch1 = df.agg(kll_sketch_agg_bigint($"value")).collect()(0)(0) + assert(sketch1 != null) + assert(sketch1.asInstanceOf[Array[Byte]].length > 0) + + // Test with explicit k + val sketch2 = df.agg(kll_sketch_agg_bigint($"value", 400)).collect()(0)(0) + assert(sketch2 != null) + assert(sketch2.asInstanceOf[Array[Byte]].length > 0) + + // Test with column name + val sketch3 = df.agg(kll_sketch_agg_bigint("value")).collect()(0)(0) + assert(sketch3 != null) + } + + test("kll_sketch_agg_float basic functionality") { + val df = Seq(1.0f, 2.0f, 3.0f, 4.0f, 5.0f).toDF("value") + + val sketch = df.agg(kll_sketch_agg_float($"value")).collect()(0)(0) + assert(sketch != null) + assert(sketch.asInstanceOf[Array[Byte]].length > 0) + + // Test with k parameter + val sketch2 = df.agg(kll_sketch_agg_float($"value", 300)).collect()(0)(0) + assert(sketch2 != null) + } + + test("kll_sketch_agg_double basic functionality") { + val df = Seq(1.0, 2.0, 3.0, 4.0, 5.0).toDF("value") + + val sketch = df.agg(kll_sketch_agg_double($"value")).collect()(0)(0) + assert(sketch != null) + assert(sketch.asInstanceOf[Array[Byte]].length > 0) + } + + test("kll_sketch_to_string functions") { + val df = Seq(1, 2, 3, 4, 5).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + val result = sketchDf.select(kll_sketch_to_string_bigint($"sketch")).collect()(0)(0) + assert(result != null) + assert(result.asInstanceOf[String].length > 0) + assert(result.asInstanceOf[String].contains("Kll")) + } + + test("kll_sketch_get_n functions") { + val df = Seq(1, 2, 3, 4, 5).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + val n = sketchDf.select(kll_sketch_get_n_bigint($"sketch")).collect()(0)(0) + assert(n == 5L) + } + + test("kll_sketch_merge_bigint") { + val df = Seq(1, 2, 3).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + val merged = sketchDf.select( + kll_sketch_merge_bigint($"sketch", $"sketch").alias("merged") + ).collect()(0)(0) + assert(merged != null) + assert(merged.asInstanceOf[Array[Byte]].length > 0) + } + + test("kll_sketch_get_quantile_bigint") { + val df = Seq(1, 2, 3, 4, 5).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + val quantile = sketchDf.select( + kll_sketch_get_quantile_bigint($"sketch", lit(0.5)) + ).collect()(0)(0) + assert(quantile.asInstanceOf[Long] >= 1 && quantile.asInstanceOf[Long] <= 5) + + // Test with array of ranks + val quantiles = sketchDf.select( + kll_sketch_get_quantile_bigint($"sketch", array(lit(0.25), lit(0.5), lit(0.75))) + ).collect()(0)(0) + assert(quantiles != null) + } + + test("kll_sketch_get_rank_bigint") { + val df = Seq(1, 2, 3, 4, 5).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + val rank = sketchDf.select( + kll_sketch_get_rank_bigint($"sketch", lit(3)) + ).collect()(0)(0) + assert(rank.asInstanceOf[Double] >= 0.0 && rank.asInstanceOf[Double] <= 1.0) + } + + test("kll_sketch float variants") { + val df = Seq(1.0f, 2.0f, 3.0f, 4.0f, 5.0f).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_float($"value").alias("sketch")) + + // Test to_string + val str = sketchDf.select(kll_sketch_to_string_float($"sketch")).collect()(0)(0) + assert(str.asInstanceOf[String].contains("Kll")) + + // Test get_n + val n = sketchDf.select(kll_sketch_get_n_float($"sketch")).collect()(0)(0) + assert(n == 5L) + + // Test merge + val merged = sketchDf.select( + kll_sketch_merge_float($"sketch", $"sketch") + ).collect()(0)(0) + assert(merged != null) + + // Test get_quantile + val quantile = sketchDf.select( + kll_sketch_get_quantile_float($"sketch", lit(0.5)) + ).collect()(0)(0) + assert(quantile != null) + + // Test get_rank + val rank = sketchDf.select( + kll_sketch_get_rank_float($"sketch", lit(3.0f)) + ).collect()(0)(0) + assert(rank.asInstanceOf[Double] >= 0.0 && rank.asInstanceOf[Double] <= 1.0) + } + + test("kll_sketch double variants") { + val df = Seq(1.0, 2.0, 3.0, 4.0, 5.0).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_double($"value").alias("sketch")) + + // Test to_string + val str = sketchDf.select(kll_sketch_to_string_double($"sketch")).collect()(0)(0) + assert(str.asInstanceOf[String].contains("Kll")) + + // Test get_n + val n = sketchDf.select(kll_sketch_get_n_double($"sketch")).collect()(0)(0) + assert(n == 5L) + + // Test merge + val merged = sketchDf.select( + kll_sketch_merge_double($"sketch", $"sketch") + ).collect()(0)(0) + assert(merged != null) + + // Test get_quantile + val quantile = sketchDf.select( + kll_sketch_get_quantile_double($"sketch", lit(0.5)) + ).collect()(0)(0) + assert(quantile != null) + + // Test get_rank + val rank = sketchDf.select( + kll_sketch_get_rank_double($"sketch", lit(3.0)) + ).collect()(0)(0) + assert(rank.asInstanceOf[Double] >= 0.0 && rank.asInstanceOf[Double] <= 1.0) + } + + test("kll_sketch with null values") { + val df = Seq(Some(1), None, Some(3), Some(4), None).toDF("value") + val sketchDf = df.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + val n = sketchDf.select(kll_sketch_get_n_bigint($"sketch")).collect()(0)(0) + // Should only count non-null values + assert(n == 3L) + } } case class B(c: Option[Double]) From aad0f0010c8a4f2091286db668781bf53ddb5e51 Mon Sep 17 00:00:00 2001 From: VINDHYA G BHAT Date: Tue, 11 Nov 2025 09:03:07 -0800 Subject: [PATCH 099/400] [SPARK-53448][PYTHON] Conversion of a pyspark DataFrame with a Variant column to pandas fails with an error ### What changes were proposed in this pull request? When a dataframe is converted to Pandas, VariantVal is sent to the conversion function which is not checked resulting to error. This PR aims to fix it by returning the VariantVal value itself. ### Why are the changes needed? df_variant = df.withColumn("variant_column", expr("parse_json(json_data)")) pdf = df_variant.toPandas() Above code returns error if there is no check for VariantVal value in convert varianttype method. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tested locally by doing the changes and running the code ### Was this patch authored or co-authored using generative AI tooling? No Closes #52260 from VindhyaG/SPARK-53448. Lead-authored-by: VINDHYA G BHAT Co-authored-by: Vindhya G Signed-off-by: Hyukjin Kwon (cherry picked from commit 9ff0fba06e758d2509dd8eb7a38b01cc8720d43d) Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/pandas/types.py | 6 ++++-- python/pyspark/sql/tests/test_types.py | 17 +++++++++++++++++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index d8a45daa77e89..9583c8ac72888 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -23,6 +23,8 @@ import itertools from typing import Any, Callable, Iterable, List, Optional, Union, TYPE_CHECKING +from pyspark.errors import PySparkTypeError, UnsupportedOperationException, PySparkValueError +from pyspark.loose_version import LooseVersion from pyspark.sql.types import ( cast, BooleanType, @@ -56,8 +58,6 @@ Geography, _create_row, ) -from pyspark.errors import PySparkTypeError, UnsupportedOperationException, PySparkValueError -from pyspark.loose_version import LooseVersion if TYPE_CHECKING: import pandas as pd @@ -1157,6 +1157,8 @@ def convert_udt(value: Any) -> Any: elif isinstance(dt, VariantType): def convert_variant(value: Any) -> Any: + if isinstance(value, VariantVal): + return value if ( isinstance(value, dict) and all(key in value for key in ["value", "metadata"]) diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 4ff2ab3e5cd73..0a5219202a3a8 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -2452,6 +2452,23 @@ def test_variant_type(self): with self.assertRaises(PySparkValueError, msg="Rows cannot be of type VariantVal"): self.spark.createDataFrame([VariantVal.parseJson("2")], "v variant") + def test_variant_to_pandas(self): + import pandas as pd + import json + + expected_values = [ + ("str", '"%s"' % ("0123456789" * 10), "0123456789" * 10), + ("short_str", '"abc"', "abc"), + ] + json_str = "{%s}" % ",".join(['"%s": %s' % (t[0], t[1]) for t in expected_values]) + df = self.spark.createDataFrame([({"json": json_str})]) + df_variant = df.select(F.parse_json(df.json).alias("v")) + pandas = df_variant.toPandas() + test_record = json.loads(pandas["v"].iloc[0].toJson()) + self.assertIsInstance(pandas, pd.DataFrame) + self.assertEqual(expected_values[0][2], test_record["str"]) + self.assertEqual(expected_values[1][2], test_record["short_str"]) + def test_geospatial_encoding(self): df = self.spark.createDataFrame( [ From 29d54ccc11577ef8927a6f6a49f5a4fb49e2003c Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sun, 9 Nov 2025 23:15:59 -0800 Subject: [PATCH 100/400] [SPARK-54259][CORE] Downgrade "Shutdown hook called" messages from INFO to DEBUG ### What changes were proposed in this pull request? Downgrades messages like these from INFO to DEBUG: ``` 25/11/09 06:18:58 INFO ShutdownHookManager: Shutdown hook called 25/11/09 06:18:58 INFO ShutdownHookManager: Deleting directory /private/var/folders/1v/dqhbgmt10vl6v3tdlwvvx90r0000gp/T/spark-23f1af4f-eba9-47d1-a284-d7af223c59fb ``` ### Why are the changes needed? Shutdown hook invocation messages are irrelevant to most users but show up on every invocation of `spark-submit`. ### Does this PR introduce _any_ user-facing change? The messages shown above will no longer show up if minimum log severity is set to INFO. ### How was this patch tested? Ran spark-submit and spark-pipelines. ### Was this patch authored or co-authored using generative AI tooling? Closes #52955 from sryza/shutdown-hook. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 6211f7be5913e0f36330ff028bc651a3889949c1) Signed-off-by: Sandy Ryza --- .../scala/org/apache/spark/util/ShutdownHookManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index af93f781343d2..6a6b0299a0bcd 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -61,12 +61,12 @@ private[spark] object ShutdownHookManager extends Logging { // Add a shutdown hook to delete the temp dirs when the JVM exits logDebug("Adding shutdown hook") // force eager creation of logger addShutdownHook(TEMP_DIR_SHUTDOWN_PRIORITY) { () => - logInfo("Shutdown hook called") + logDebug("Shutdown hook called") // we need to materialize the paths to delete because deleteRecursively removes items from // shutdownDeletePaths as we are traversing through it. shutdownDeletePaths.toArray.foreach { dirPath => try { - logInfo(log"Deleting directory ${MDC(LogKeys.PATH, dirPath)}") + logDebug(log"Deleting directory ${MDC(LogKeys.PATH, dirPath)}") Utils.deleteRecursively(new File(dirPath)) } catch { case e: Exception => From b9fcfde3d678f2d8108774308d090c256ae62775 Mon Sep 17 00:00:00 2001 From: Dylan Wong Date: Tue, 11 Nov 2025 17:04:05 -0800 Subject: [PATCH 101/400] [SPARK-54290][SS] Skip checksum creation if path already exists without checksum ### What changes were proposed in this pull request? PR for https://github.com/apache/spark/pull/52985 in [branch-4.1](https://github.com/apache/spark/tree/branch-4.1). This PR modifies ChecksumCheckpointFileManager to fall back to the underlying CheckpointFileManager when the target path exists but the checksum file does not. This prevents failures in checkpoint recovery scenarios where checksum validation cannot be performed. ### Why are the changes needed? Consider the case using STATE_STORE_CHECKPOINT_FORMAT_VERSION = 1 when a batch fails but state files are written. If on the next run, we try to upload both a new state file and a file checksum, the file could fail to be uploaded but the file checksum is uploaded successfully. This would lead to a situation where the old file could be loaded and compared with the new file checksum, which would fail the checksum verification. This issue does not happen when STATE_STORE_CHECKPOINT_FORMAT_VERSION = 2 since each batch run unique ids will be created. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Added unit tests in ChecksumCheckpointFileManagerSuite to verify fallback behavior - Added a failure injection test in RocksDBCheckpointFailureInjectionSuite to simulate how this error is caused ### Was this patch authored or co-authored using generative AI tooling? No Closes #53000 from dylanwong250/SPARK-54290-4.1. Authored-by: Dylan Wong Signed-off-by: Anish Shrigondekar --- .../apache/spark/sql/internal/SQLConf.scala | 13 + .../ChecksumCheckpointFileManager.scala | 30 ++- .../state/HDFSBackedStateStoreProvider.scala | 4 +- .../execution/streaming/state/RocksDB.scala | 21 +- .../streaming/state/RocksDBFileManager.scala | 5 +- .../streaming/state/StateStoreConf.scala | 14 + .../ChecksumCheckpointFileManagerSuite.scala | 80 +++++- ...ailureInjectionCheckpointFileManager.scala | 6 +- ...cksDBCheckpointFailureInjectionSuite.scala | 239 +++++++++++++++++- 9 files changed, 384 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 2316acaa173e8..c63fc2d92743d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3453,6 +3453,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val STREAMING_CHECKPOINT_FILE_CHECKSUM_SKIP_CREATION_IF_FILE_MISSING_CHECKSUM = + buildConf("spark.sql.streaming.checkpoint.fileChecksum.skipCreationIfFileMissingChecksum") + .internal() + .doc("When true, if a microbatch is retried, if a file already exists but its checksum " + + "file does not exist, the file checksum will not be created. This is useful for " + + "compatibility with files created before file checksums were enabled.") + .version("4.2.0") + .booleanConf + .createWithDefault(true) + val PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION = buildConf("spark.sql.statistics.parallelFileListingInStatsComputation.enabled") .internal() @@ -6779,6 +6789,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def checkpointFileChecksumEnabled: Boolean = getConf(STREAMING_CHECKPOINT_FILE_CHECKSUM_ENABLED) + def checkpointFileChecksumSkipCreationIfFileMissingChecksum: Boolean = + getConf(STREAMING_CHECKPOINT_FILE_CHECKSUM_SKIP_CREATION_IF_FILE_MISSING_CHECKSUM) + def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED) def useDeprecatedKafkaOffsetFetching: Boolean = getConf(USE_DEPRECATED_KAFKA_OFFSET_FETCHING) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/ChecksumCheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/ChecksumCheckpointFileManager.scala index 7f801392c2f4a..637d11ad890bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/ChecksumCheckpointFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/ChecksumCheckpointFileManager.scala @@ -133,11 +133,21 @@ case class ChecksumFile(path: Path) { * number of threads using file manager * 2. * Setting this differently can lead to file operation being blocked waiting for * a free thread. + * @param skipCreationIfFileMissingChecksum (ES-1629547): If true, when a file already exists + * but its checksum file does not exist, fall back to using the underlying + * file manager directly instead of creating with checksum. This is useful + * for compatibility with files created before checksums were enabled. Consider + * the case when a batch fails but state files are written. If on the next run, + * we try to upload both a new file and a checksum file, the file could fail to be + * uploaded but the checksum file is uploaded successfully. This would lead to a + * situation where the old file could be loaded and compared with the new file + * checksum, which would fail the checksum verification. */ class ChecksumCheckpointFileManager( private val underlyingFileMgr: CheckpointFileManager, val allowConcurrentDelete: Boolean = false, - val numThreads: Int) + val numThreads: Int, + val skipCreationIfFileMissingChecksum: Boolean) extends CheckpointFileManager with Logging { assert(numThreads % 2 == 0, "numThreads must be a multiple of 2, we need 1 for the main file" + "and another for the checksum file") @@ -160,9 +170,18 @@ class ChecksumCheckpointFileManager( underlyingFileMgr.mkdirs(path) } + private def shouldSkipChecksumCreation(path: Path): Boolean = { + skipCreationIfFileMissingChecksum && + underlyingFileMgr.exists(path) && !underlyingFileMgr.exists(getChecksumPath(path)) + } + override def createAtomic(path: Path, overwriteIfPossible: Boolean): CancellableFSDataOutputStream = { - createWithChecksum(path, underlyingFileMgr.createAtomic(_, overwriteIfPossible)) + if (shouldSkipChecksumCreation(path)) { + underlyingFileMgr.createAtomic(path, overwriteIfPossible) + } else { + createWithChecksum(path, underlyingFileMgr.createAtomic(_, overwriteIfPossible)) + } } private def createWithChecksum(path: Path, @@ -327,8 +346,11 @@ class ChecksumFSDataInputStream( override def close(): Unit = { if (!closed) { // We verify the checksum only when the client is done reading. - verifyChecksum() - closeInternal() + try { + verifyChecksum() + } finally { + closeInternal() + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index aa4fa9bfaf627..9dc236e0ffd44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -466,7 +466,9 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with // (one for main file and another for checksum file). // Since this fm is used by both query task and maintenance thread, // then we need 2 * 2 = 4 threads. - numThreads = 4) + numThreads = 4, + skipCreationIfFileMissingChecksum = + storeConf.checkpointFileChecksumSkipCreationIfFileMissingChecksum) } else { mgr } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index fb3ef606b8f34..aa02d708933d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -150,20 +150,29 @@ class RocksDB( localTempDir: File, hadoopConf: Configuration, codecName: String, - loggingId: String): RocksDBFileManager = { + loggingId: String, + storeConf: StateStoreConf): RocksDBFileManager = { new RocksDBFileManager( dfsRootDir, localTempDir, hadoopConf, codecName, loggingId = loggingId, + storeConf, fileChecksumEnabled = conf.fileChecksumEnabled, fileChecksumThreadPoolSize = fileChecksumThreadPoolSize ) } - private[spark] val fileManager = createFileManager(dfsRootDir, createTempDir("fileManager"), - hadoopConf, conf.compressionCodec, loggingId = loggingId) + private[spark] val fileManager = createFileManager( + dfsRootDir, + createTempDir("fileManager"), + hadoopConf, + conf.compressionCodec, + loggingId = loggingId, + storeConf = conf.stateStoreConf + ) + private val byteArrayPair = new ByteArrayPair() private val commitLatencyMs = new mutable.HashMap[String, Long]() @@ -2067,7 +2076,8 @@ case class RocksDBConf( compression: String, reportSnapshotUploadLag: Boolean, fileChecksumEnabled: Boolean, - maxVersionsToDeletePerMaintenance: Int) + maxVersionsToDeletePerMaintenance: Int, + stateStoreConf: StateStoreConf) object RocksDBConf { /** Common prefix of all confs in SQLConf that affects RocksDB */ @@ -2267,7 +2277,8 @@ object RocksDBConf { getStringConf(COMPRESSION_CONF), storeConf.reportSnapshotUploadLag, storeConf.checkpointFileChecksumEnabled, - storeConf.maxVersionsToDeletePerMaintenance) + storeConf.maxVersionsToDeletePerMaintenance, + storeConf) } def apply(): RocksDBConf = apply(new StateStoreConf()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 2e86ff70d58fb..ed34a46889d85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -132,6 +132,7 @@ class RocksDBFileManager( hadoopConf: Configuration, codecName: String = CompressionCodec.ZSTD, loggingId: String = "", + storeConf: StateStoreConf = StateStoreConf.empty, fileChecksumEnabled: Boolean = false, fileChecksumThreadPoolSize: Option[Int] = None) extends Logging { @@ -149,7 +150,9 @@ class RocksDBFileManager( mgr, // Allowing this for perf, since we do orphan checksum file cleanup in maintenance anyway allowConcurrentDelete = true, - numThreads = fileChecksumThreadPoolSize.get) + numThreads = fileChecksumThreadPoolSize.get, + skipCreationIfFileMissingChecksum + = storeConf.checkpointFileChecksumSkipCreationIfFileMissingChecksum) } else { mgr } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala index 74904a37f4504..3e190eedc9f44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala @@ -114,6 +114,20 @@ class StateStoreConf( val enableStateStoreCheckpointIds = StatefulOperatorStateInfo.enableStateStoreCheckpointIds(sqlConf) + /** + * Whether to skip checksum creation if file missing checksum. + * + * Consider the case using STATE_STORE_CHECKPOINT_FORMAT_VERSION = 1 when a batch fails but state + * files are written. If on the next run, we try to upload both a new state file and a file + * checksum, the file could fail to be uploaded but the file checksum is uploaded successfully. + * This would lead to a situation where the old file could be loaded and compared with the new + * file checksum, which would fail the checksum verification. This issue does not happen when + * STATE_STORE_CHECKPOINT_FORMAT_VERSION = 2 since each batch run unique ids will be created. + */ + val checkpointFileChecksumSkipCreationIfFileMissingChecksum: Boolean = + sqlConf.checkpointFileChecksumSkipCreationIfFileMissingChecksum && + !enableStateStoreCheckpointIds + /** * Whether the coordinator is reporting state stores trailing behind in snapshot uploads. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ChecksumCheckpointFileManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ChecksumCheckpointFileManagerSuite.scala index 29d09f5d52f9d..b15e8f167db58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ChecksumCheckpointFileManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ChecksumCheckpointFileManagerSuite.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs._ import org.apache.spark.SparkException import org.apache.spark.sql.execution.streaming.checkpointing._ +import org.apache.spark.sql.internal.SQLConf /** * This inherits tests for the [[CheckpointFileManager]] from [[CheckpointFileManagerTests]]. @@ -57,9 +58,27 @@ abstract class ChecksumCheckpointFileManagerSuite extends CheckpointFileManagerT s"expected main files: $mainFilesForExistingChecksumFiles / actual files: $files") } + override def createManager(path: Path): CheckpointFileManager = { + createChecksumManager( + path, + skipCreationIfFileMissingChecksum = + SQLConf.STREAMING_CHECKPOINT_FILE_CHECKSUM_SKIP_CREATION_IF_FILE_MISSING_CHECKSUM + .defaultValue.get) + } + /** Create a normal CheckpointFileManager (not the checksum checkpoint manager) */ protected def createNoChecksumManager(path: Path): CheckpointFileManager + protected def createChecksumManager( + path: Path, + skipCreationIfFileMissingChecksum: Boolean): CheckpointFileManager = { + new ChecksumCheckpointFileManager( + createNoChecksumManager(path), + allowConcurrentDelete = true, + numThreads = 4, + skipCreationIfFileMissingChecksum = skipCreationIfFileMissingChecksum) + } + private def makeDir(fm: CheckpointFileManager, dir: Path): Unit = { assert(!fm.exists(dir)) fm.mkdirs(dir) @@ -184,29 +203,62 @@ abstract class ChecksumCheckpointFileManagerSuite extends CheckpointFileManagerT assert(regularFm.open(path).readContent() == content) } } -} -class FileContextChecksumCheckpointFileManagerSuite extends ChecksumCheckpointFileManagerSuite { - override def createManager(path: Path): CheckpointFileManager = { - new ChecksumCheckpointFileManager( - createNoChecksumManager(path), - allowConcurrentDelete = true, - numThreads = 4) + test("skip checksum creation if file missing checksum") { + withTempHadoopPath { basePath => + val regularFm = createNoChecksumManager(basePath) + // Mkdirs + val dir = new Path(s"$basePath/dir/subdir/subsubdir") + makeDir(regularFm, dir) + + // Create a file using the regular file manager + val path = new Path(s"$dir/file") + regularFm.createAtomic(path, overwriteIfPossible = true).writeContent(content).close() + assert(regularFm.exists(path)) + + // Now try to write and read the file with the checksum manager with fallback. + val checksumFmWithFallback = + createChecksumManager(basePath, skipCreationIfFileMissingChecksum = true) + // Overwrite the file with a different content. + checksumFmWithFallback.createAtomic( + path, overwriteIfPossible = true).writeContent(content + 1).close() + assert(checksumFmWithFallback.open(path).readContent() == content + 1) + // Checksum should not be created since we fallback to the underlying file manager. + assert(!checksumFmWithFallback.exists(getChecksumPath(path))) + + // Now try to write and read the file with the checksum manager without fallback. + val checksumFmWithoutFallback = + createChecksumManager(basePath, skipCreationIfFileMissingChecksum = false) + // Overwrite the file with a different content. + checksumFmWithoutFallback.createAtomic( + path, overwriteIfPossible = true).writeContent(content + 2).close() + assert(checksumFmWithoutFallback.open(path).readContent() == content + 2) + // Checksum should be created since we don't fallback to the underlying file manager. + assert(checksumFmWithoutFallback.exists(getChecksumPath(path))) + + // Try to write and read the file with the checksum manager with fallback when the checksum + // file already exists. + checksumFmWithFallback.createAtomic( + path, overwriteIfPossible = true).writeContent(content + 3).close() + // This read should succeed since we do not fallback to the underlying file manager, since + // the checksum file already exists. + assert(checksumFmWithFallback.open(path).readContent() == content + 3) + assert(checksumFmWithFallback.exists(getChecksumPath(path))) + + regularFm.close() + checksumFmWithFallback.close() + checksumFmWithoutFallback.close() + } } +} +class FileContextChecksumCheckpointFileManagerSuite extends ChecksumCheckpointFileManagerSuite { protected def createNoChecksumManager(path: Path): CheckpointFileManager = { new FileContextBasedCheckpointFileManager(path, new Configuration()) } } class FileSystemChecksumCheckpointFileManagerSuite extends ChecksumCheckpointFileManagerSuite { - override def createManager(path: Path): CheckpointFileManager = { - new ChecksumCheckpointFileManager( - createNoChecksumManager(path), - allowConcurrentDelete = true, - numThreads = 4) - } - protected def createNoChecksumManager(path: Path): CheckpointFileManager = { new FileSystemBasedCheckpointFileManager(path, new Configuration()) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala index fad207f97dd9f..898e324a954e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala @@ -307,7 +307,8 @@ object FailureInjectionRocksDBStateStoreProvider { localTempDir: File, hadoopConf: Configuration, codecName: String, - loggingId: String): RocksDBFileManager = { + loggingId: String, + storeConf: StateStoreConf): RocksDBFileManager = { new RocksDBFileManager( dfsRootDir, localTempDir, @@ -315,7 +316,8 @@ object FailureInjectionRocksDBStateStoreProvider { codecName, loggingId = loggingId, fileChecksumEnabled = this.conf.fileChecksumEnabled, - fileChecksumThreadPoolSize = this.fileChecksumThreadPoolSize) { + fileChecksumThreadPoolSize = this.fileChecksumThreadPoolSize, + storeConf = this.conf.stateStoreConf) { override def getFileSystem( myDfsRootDir: String, myHadoopConf: Configuration): FileSystem = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala index 30f1d77441bfd..0b9690ee72775 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.execution.streaming.runtime.MemoryStream -import org.apache.spark.sql.functions.count +import org.apache.spark.sql.functions.{count, udf} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS import org.apache.spark.sql.streaming._ @@ -587,6 +587,243 @@ class RocksDBCheckpointFailureInjectionSuite extends StreamTest } } + case class FailureConf3( + skipCreationIfFileMissingChecksum: Boolean, + checkpointFormatVersion : String) { + override def toString: String = { + s"skipCreationIfFileMissingChecksum = $skipCreationIfFileMissingChecksum, " + + s"checkpointFormatVersion = $checkpointFormatVersion" + } + } + + private def versionsPresent(dir: File, suffix: String): Seq[(Long, Option[String])] = { + dir.listFiles.filter(_.getName.endsWith(suffix)) + .filter(!_.getName.startsWith(".")) + .map(_.getName.stripSuffix(suffix).split("_")) + .map { + case Array(version, uniqueId) => (version.toLong, Some(uniqueId)) + case Array(version) => (version.toLong, None) + } + .sorted + .distinct + .toSeq + } + + /** + * Test that verifies upgrading from checksum disabled to checksum enabled after state files are + * written but before batch commit completes. The important part of this test is that files are + * not overwritten if they already exist. When checkpointFormatVersion is 2, we will not run into + * the checksum verification failure because each batch run uses unique changelog file names. + * + * Scenario: + * 1. Start with checksum verification disabled + * 2. Run batch 1 successfully (writes 1.changelog without .crc) + * 3. Start batch 2 - state store commits successfully (writes 2.changelog without .crc) but the + * batch fails before the commit is complete (via UDF exception). This leaves 2.changelog on + * disk without a corresponding commit log file + * 4. Restart query with checksum verification enabled and with a query where the changelog file + * contents will change from batch 2 + * 5. Run batch 2 again and it succeeds and writes 2.changelog (and 2.changelog.crc if + * STREAMING_CHECKPOINT_FILE_CHECKSUM_SKIP_CREATION_IF_FILE_MISSING_CHECKSUM is disabled) + * 6. Run batch 3 and it succeeds and writes 3.changelog and 3.changelog.crc + * 7. Query starts with STREAMING_CHECKPOINT_FILE_CHECKSUM_SKIP_CREATION_IF_FILE_MISSING_CHECKSUM + * enabled/disabled and the different behavior is shown in the test + */ + + Seq( + FailureConf3(skipCreationIfFileMissingChecksum = false, checkpointFormatVersion = "1"), + FailureConf3(skipCreationIfFileMissingChecksum = true, checkpointFormatVersion = "1"), + FailureConf3(skipCreationIfFileMissingChecksum = false, checkpointFormatVersion = "2"), + FailureConf3(skipCreationIfFileMissingChecksum = true, checkpointFormatVersion = "2") + ).foreach { failureConf => + test(s"Upgrading from file checksum disabled to enabled " + + "after state commits without batch commit " + failureConf.toString) { + val hadoopConf = new Configuration() + hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fileManagerClassName) + val rocksdbChangelogCheckpointingConfKey = + RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + ".changelogCheckpointing.enabled" + + withTempDirAllowFailureInjection { (checkpointDir, injectionState) => + var forceTaskFailure = false + val failUDF = udf((value: Int) => { + if (forceTaskFailure) { + // This will fail all close() call to trigger query failures in execution phase. + throw new RuntimeException("Ingest task failure") + } + value + }) + + val inputData = MemoryStream[Int] + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*").as("count")) + // would fail here after writing the changelog file for the agg + .select(failUDF($"value").as("value"), $"count") + .as[(Int, Long)] + + val aggregated2 = + inputData.toDF() + .select($"value" + 1000 as "value") // This is to make the changelog file different + .groupBy($"value") + .agg(count("*").as("count")) + // would fail here after writing the changelog file for the agg + .select(failUDF($"value").as("value"), $"count") + .as[(Int, Long)] + + def getRunConf(checksumEnabled: Boolean) : Map[String, String] = { + Map( + rocksdbChangelogCheckpointingConfKey -> "true", + SQLConf.STATE_STORE_CHECKPOINT_FORMAT_VERSION.key -> + failureConf.checkpointFormatVersion, + SQLConf.STREAMING_CHECKPOINT_FILE_CHECKSUM_ENABLED.key -> checksumEnabled.toString, + SQLConf.STREAMING_CHECKPOINT_FILE_CHECKSUM_SKIP_CREATION_IF_FILE_MISSING_CHECKSUM.key -> + failureConf.skipCreationIfFileMissingChecksum.toString, + STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key -> fileManagerClassName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") + } + + // Verify that the changelog files exists for a version + def verifyChangelogFileExists(version: Long) : Boolean = { + versionsPresent(new File(checkpointDir, "state/0/0"), ".changelog").exists { + case (v, uniqueId) => + if (failureConf.checkpointFormatVersion == "1") { + v == version && uniqueId.isEmpty + } else { + v == version && uniqueId.isDefined + } + } + } + + // Verify that the changelog checksum files exists for a version + def verifyChangelogFileChecksumExists(version: Long) : Boolean = { + versionsPresent(new File(checkpointDir, "state/0/0"), ".changelog.crc").exists { + case (v, uniqueId) => + if (failureConf.checkpointFormatVersion == "1") { + v == version && uniqueId.isEmpty + } else { + v == version && uniqueId.isDefined + } + } + } + + // First run: file checksum disabled + val firstRunConfs = getRunConf(checksumEnabled = false) + + testStream(aggregated, Update)( + StartStream( + checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = firstRunConfs), + AddData(inputData, 3), + CheckLastBatch((3, 1)), + Execute { _ => + forceTaskFailure = true + }, + AddData(inputData, 3, 2), + ExpectFailure[SparkException] { ex => + ex.getCause.getMessage.contains("FAILED_EXECUTE_UDF") + } + ) + + // Verify that the changelog file was written + assert(verifyChangelogFileExists(2)) + // Verify that the changelog file checksum was NOT written since it was disabled + assert(!verifyChangelogFileChecksumExists(2)) + + // Verify that the commit file was written + assert((new File(checkpointDir, "commits/0")).exists()) + // Verify that the commit file was NOT written + assert(!(new File(checkpointDir, "commits/1")).exists()) + + // Second run: STREAMING_CHECKPOINT_FILE_CHECKSUM_ENABLED enabled with + // allowOverwriteInRename = false. This simulates an upgrade to a new version where the + // file checksum is enabled. The allowOverwriteInRename is set to false to test the case + // when overwriting the changelog file fails. This is to simulate the case where the + // changelog file is not overwritten but the checksum file is written. + injectionState.allowOverwriteInRename = false + forceTaskFailure = false + + val secondRunConfs = getRunConf(checksumEnabled = true) + + inputData.addData(3, 1) + + // The query should restart successfully and handle files without checksums, whether + // skipCreationIfFileMissingChecksum is enabled or disabled. The problem + // arises on the load after this run. + testStream(aggregated2, Update)( + StartStream( + checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = secondRunConfs), + AddData(inputData, 4), + CheckLastBatch((1003, 2), (1001, 1), (1004, 1)), + StopStream + ) + + assert(verifyChangelogFileExists(3)) + assert(verifyChangelogFileChecksumExists(3)) + + // Verify that the commit files were written + assert((new File(checkpointDir, "commits/1")).exists()) + assert((new File(checkpointDir, "commits/2")).exists()) + + val failureCase = + !failureConf.skipCreationIfFileMissingChecksum && + failureConf.checkpointFormatVersion == "1" + + if (failureCase) { + assert(verifyChangelogFileChecksumExists(2)) + + // The query does not succeed, since we load the old changelog file with the checksum from + // the new changelog file that did not overwrite the old one. This will lead to a checksum + // verification failure when we try to load the old changelog file with the checksum from + // the new changelog file that did not overwrite the old one. + testStream(aggregated2, Update)( + StartStream( + checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = secondRunConfs), + AddData(inputData, 4), + ExpectFailure[SparkException] { ex => + ex.getMessage.contains("CHECKPOINT_FILE_CHECKSUM_VERIFICATION_FAILED") + ex.getMessage.contains("2.changelog") + } + ) + + // Verify that the commit file was not written + assert(!(new File(checkpointDir, "commits/3")).exists()) + } else { + if (failureConf.checkpointFormatVersion == "1") { + // With checkpointFormatVersion = 1, the changelog file checksum should not be written + assert(!verifyChangelogFileChecksumExists(2)) + } else { + // With checkpointFormatVersion = 2, the changelog file checksum should be written + assert(verifyChangelogFileChecksumExists(2)) + } + + // The query should restart successfully + testStream(aggregated2, Update)( + StartStream( + checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = secondRunConfs), + AddData(inputData, 4), + CheckLastBatch((1004, 2)), + StopStream + ) + + // Verify again the 2.changelog file checksum exists or not + if (failureConf.checkpointFormatVersion == "1") { + assert(!verifyChangelogFileChecksumExists(2)) + } else { + assert(verifyChangelogFileChecksumExists(2)) + } + + assert(verifyChangelogFileExists(4)) + assert(verifyChangelogFileChecksumExists(4)) + assert((new File(checkpointDir, "commits/3")).exists()) + } + } + } + } + def commitAndGetCheckpointId(db: RocksDB): Option[String] = { val (v, ci) = db.commit() ci.stateStoreCkptId From 7e4031458d62ed0d52a80d6d2846604cdd4e1131 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 11 Nov 2025 17:33:06 -0800 Subject: [PATCH 102/400] [SPARK-54261][SQL][4.1] Disable `spark.sql.scripting.enabled` by default in Apache Spark 4.1.0 ### What changes were proposed in this pull request? This PR aims to revert the following logically to disable `spark.sql.scripting.enabled` by default in Apache Spark 4.1.0 while keeping it in Apache Spark 4.2.0 in `master` branch. - #52272 ### Why are the changes needed? Given that too many open subtasks of SPARK-48338 exist, the feature is still experimental like Apache Spark 4.0.0. ### Does this PR introduce _any_ user-facing change? No because Apache Spark 4.1.0 will disable this feature by default like Apache Spark 4.0.0. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52958 from dongjoon-hyun/SPARK-54261. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../parser/SqlScriptingParserSuite.scala | 2 + .../sql/connect/ClientE2ETestSuite.scala | 88 ++++++++++--------- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 + .../ExecuteImmediateEndToEndSuite.scala | 17 ++-- .../sql/scripting/SqlScriptingE2eSuite.scala | 1 + .../SqlScriptingExecutionSuite.scala | 1 + .../SqlScriptingInterpreterSuite.scala | 2 + 8 files changed, 66 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c63fc2d92743d..6ab081d54dc47 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4304,7 +4304,7 @@ object SQLConf { "flow and error handling.") .version("4.0.0") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val SQL_SCRIPTING_CONTINUE_HANDLER_ENABLED = buildConf("spark.sql.scripting.continueHandlerEnabled") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 298329db1ee30..de4f8c4aa2cd7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -30,10 +30,12 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { protected override def beforeAll(): Unit = { super.beforeAll() + conf.setConf(SQLConf.SQL_SCRIPTING_ENABLED, true) conf.setConf(SQLConf.SQL_SCRIPTING_CONTINUE_HANDLER_ENABLED, true) } protected override def afterAll(): Unit = { + conf.unsetConf(SQLConf.SQL_SCRIPTING_ENABLED.key) conf.unsetConf(SQLConf.SQL_SCRIPTING_CONTINUE_HANDLER_ENABLED.key) super.afterAll() } diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala index 450ff8ca62490..773b4c375c0b0 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala @@ -1797,57 +1797,65 @@ class ClientE2ETestSuite // SQL Scripting tests test("SQL Script result") { - val df = spark.sql("""BEGIN - | IF 1=1 THEN - | SELECT 1; - | ELSE - | SELECT 3; - | END IF; - |END - |""".stripMargin) - checkAnswer(df, Seq(Row(1))) - } - - test("SQL Script schema") { - withTable("script_tbl") { + withSQLConf("spark.sql.scripting.enabled" -> "true") { val df = spark.sql("""BEGIN - | CREATE TABLE script_tbl (a INT, b STRING); - | INSERT INTO script_tbl VALUES (1, 'Hello'), (2, 'World'); - | SELECT * FROM script_tbl; + | IF 1=1 THEN + | SELECT 1; + | ELSE + | SELECT 3; + | END IF; |END |""".stripMargin) - assert( - df.schema == StructType( - StructField("a", IntegerType, nullable = true) - :: StructField("b", StringType, nullable = true) - :: Nil)) + checkAnswer(df, Seq(Row(1))) + } + } + + test("SQL Script schema") { + withSQLConf("spark.sql.scripting.enabled" -> "true") { + withTable("script_tbl") { + val df = spark.sql("""BEGIN + | CREATE TABLE script_tbl (a INT, b STRING); + | INSERT INTO script_tbl VALUES (1, 'Hello'), (2, 'World'); + | SELECT * FROM script_tbl; + |END + |""".stripMargin) + assert( + df.schema == StructType( + StructField("a", IntegerType, nullable = true) + :: StructField("b", StringType, nullable = true) + :: Nil)) + } } } test("SQL Script empty result") { - withTable("script_tbl") { - val df = spark.sql("""BEGIN - | CREATE TABLE script_tbl (a INT, b STRING); - | SELECT * FROM script_tbl; - |END - |""".stripMargin) - assert( - df.schema == StructType( - StructField("a", IntegerType, nullable = true) - :: StructField("b", StringType, nullable = true) - :: Nil)) - checkAnswer(df, Seq.empty) + withSQLConf("spark.sql.scripting.enabled" -> "true") { + withTable("script_tbl") { + val df = spark.sql("""BEGIN + | CREATE TABLE script_tbl (a INT, b STRING); + | SELECT * FROM script_tbl; + |END + |""".stripMargin) + assert( + df.schema == StructType( + StructField("a", IntegerType, nullable = true) + :: StructField("b", StringType, nullable = true) + :: Nil)) + checkAnswer(df, Seq.empty) + } } } test("SQL Script no result") { - withTable("script_tbl") { - val df = spark.sql("""BEGIN - | CREATE TABLE script_tbl (a INT, b STRING); - |END - |""".stripMargin) - assert(df.schema == StructType(Nil)) - checkAnswer(df, Seq.empty) + withSQLConf("spark.sql.scripting.enabled" -> "true") { + withTable("script_tbl") { + val df = spark.sql("""BEGIN + | CREATE TABLE script_tbl (a INT, b STRING); + |END + |""".stripMargin) + assert(df.schema == StructType(Nil)) + checkAnswer(df, Seq.empty) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index a57c72f5fc155..23f0144dcec52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -158,6 +158,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper // regex magic. .set("spark.test.noSerdeInExplain", "true") .set(SQLConf.SCHEMA_LEVEL_COLLATIONS_ENABLED, true) + .set(SQLConf.SQL_SCRIPTING_ENABLED, true) // SPARK-32106 Since we add SQL test 'transform.sql' will use `cat` command, // here we need to ignore it. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala index c252047b3abe5..d8c24c1ac3970 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession { @@ -38,12 +39,14 @@ class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession { } test("SQL Scripting not supported inside EXECUTE IMMEDIATE") { - val executeImmediateText = "EXECUTE IMMEDIATE 'BEGIN SELECT 1; END'" - checkError( - exception = intercept[AnalysisException ] { - spark.sql(executeImmediateText) - }, - condition = "SQL_SCRIPT_IN_EXECUTE_IMMEDIATE", - parameters = Map("sqlString" -> "BEGIN SELECT 1; END")) + withSQLConf(SQLConf.SQL_SCRIPTING_ENABLED.key -> "true") { + val executeImmediateText = "EXECUTE IMMEDIATE 'BEGIN SELECT 1; END'" + checkError( + exception = intercept[AnalysisException ] { + spark.sql(executeImmediateText) + }, + condition = "SQL_SCRIPT_IN_EXECUTE_IMMEDIATE", + parameters = Map("sqlString" -> "BEGIN SELECT 1; END")) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala index 02b4a19a44ad3..db561d95058e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala @@ -69,6 +69,7 @@ class SqlScriptingE2eSuite extends QueryTest with SharedSparkSession { override protected def sparkConf: SparkConf = { super.sparkConf .set(SQLConf.ANSI_ENABLED.key, "true") + .set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true") } // Tests diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala index d911862509aa0..e26815c6c01e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala @@ -50,6 +50,7 @@ class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession { override protected def sparkConf: SparkConf = { super.sparkConf .set(SQLConf.ANSI_ENABLED.key, "true") + .set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true") } // Helpers diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 6671b52381c21..343f334f8f56c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -40,10 +40,12 @@ class SqlScriptingInterpreterSuite protected override def beforeAll(): Unit = { super.beforeAll() + conf.setConf(SQLConf.SQL_SCRIPTING_ENABLED, true) conf.setConf(SQLConf.SQL_SCRIPTING_CONTINUE_HANDLER_ENABLED, true) } protected override def afterAll(): Unit = { + conf.unsetConf(SQLConf.SQL_SCRIPTING_ENABLED.key) conf.unsetConf(SQLConf.SQL_SCRIPTING_CONTINUE_HANDLER_ENABLED.key) super.afterAll() } From cbc056ca166c5a9b52a7a30c5a721064f7dcea16 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 11 Nov 2025 17:42:17 -0800 Subject: [PATCH 103/400] [SPARK-54294][CONNECT] Normalize Connect server printed IP address ### What changes were proposed in this pull request? As title, normalize Connect server printed IP address, mostly for IPv6 address cases. ### Why are the changes needed? Start Connect server without configuring the binding host, it binds any host and reports the IPv6 anyhost on my MacBook. ``` $ SPARK_NO_DAEMONIZE=1 sbin/start-connect-server.sh ... 25/11/11 02:03:49 INFO SparkConnectServer: Spark Connect server started at: 0:0:0:0:0:0:0:0:15002 ``` while this is illegal for `java.net.URI`, which is used by Spark Connect JVM client to parse the connection url. ``` $ bin/spark-shell --remote sc://0:0:0:0:0:0:0:0:15002 WARNING: Using incubator modules: jdk.incubator.vector Using Spark's default log4j profile: org/apache/spark/log4j2-defaults.properties 25/11/11 10:56:51 WARN Utils: Your hostname, H27212-MAC-01.local, resolves to a loopback address: 127.0.0.1; using 10.242.159.140 instead (on interface en0) 25/11/11 10:56:51 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address Spark Connect REPL Host for connection URI must be defined. Options: --remote REMOTE URI of the Spark Connect Server to connect to. --host HOST Host where the Spark Connect Server is running. --port PORT Port where the Spark Connect Server is running. --use_ssl Connect to the server using SSL. --token TOKEN Token to use for authentication. --user_id USER_ID Id of the user connecting. --user_name USER_NAME Name of the user connecting. --user_agent USER_AGENT The User-Agent Client information (only intended for logging purposes by the server). --session_id SESSION_ID Session Id of the user connecting. --grpc_max_message_size SIZE Maximum message size allowed for gRPC messages in bytes. --option KEY=VALUE Key-value pair that is used to further configure the session. 25/11/11 10:56:51 INFO ShutdownHookManager: Shutdown hook called 25/11/11 10:56:51 INFO ShutdownHookManager: Deleting directory /private/var/folders/kz/qv5g7w5s1rgb_9d4mpym2v1h0000gn/T/spark-e7766d62-d66d-4a04-8c18-eeee332eedf4 ``` ### Does this PR introduce _any_ user-facing change? Only affects logs. ### How was this patch tested? before ``` ... 25/11/11 02:03:49 INFO SparkConnectServer: Spark Connect server started at: 0:0:0:0:0:0:0:0:15002 ``` `bin/spark-shell --remote sc://0:0:0:0:0:0:0:0:15002` does not work. after ``` ... 25/11/11 02:31:15 INFO SparkConnectServer: Spark Connect server started at: [::]:15002 ``` `bin/spark-shell --remote sc://[::]:15002` works ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52989 from pan3793/SPARK-54294. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 8f9385a8b9c07c30b46ac748966ba7304e8b4da9) Signed-off-by: Dongjoon Hyun --- .../spark/sql/connect/service/SparkConnectServer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala index 5b1034a4a27b7..1b2130a0e66b5 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala @@ -21,6 +21,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HOST, PORT} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Utils /** * The Spark Connect server @@ -38,9 +39,9 @@ object SparkConnectServer extends Logging { try { SparkConnectService.start(session.sparkContext) val isa = SparkConnectService.bindingAddress + val host = Utils.normalizeIpIfNeeded(isa.getAddress.getHostAddress) logInfo( - log"Spark Connect server started at: " + - log"${MDC(HOST, isa.getAddress.getHostAddress)}:${MDC(PORT, isa.getPort)}") + log"Spark Connect server started at: ${MDC(HOST, host)}:${MDC(PORT, isa.getPort)}") } catch { case e: Exception => logError("Error starting Spark Connect server", e) From dfb0e0164f0a1a4e94cd1f575d3532c69f16b0c4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 11 Nov 2025 17:48:44 -0800 Subject: [PATCH 104/400] [SPARK-54295][INFRA] Remove redundant ASFLv2 full-text in `NOTICE-binary` ### What changes were proposed in this pull request? ASFLv2 full-text carried from upstreams can be dismissed according to the ASF policy. ### Why are the changes needed? Make NOTICE a little bit not 'TL;DR' ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Passing GA ### Was this patch authored or co-authored using generative AI tooling? ni Closes #52990 from yaooqinn/SPARK-54295. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit 067678a903924890aca8be764a64b1968039ff32) Signed-off-by: Dongjoon Hyun --- NOTICE-binary | 210 -------------------------------------------------- 1 file changed, 210 deletions(-) diff --git a/NOTICE-binary b/NOTICE-binary index a3f302b1cb04d..4fb47cb8a24ab 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -755,10 +755,6 @@ project. The following notice covers the Felix files: I. Included Software - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - Licensed under the Apache License 2.0. - This product includes software developed at The OSGi Alliance (http://www.osgi.org/). Copyright (c) OSGi Alliance (2000, 2007). @@ -1071,199 +1067,6 @@ Copyright 2019 The Apache Software Foundation Hive Storage API Copyright 2018 The Apache Software Foundation - - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - Copyright 2015-2015 DataNucleus - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - Android JSON library Copyright (C) 2010 The Android Open Source Project @@ -1273,9 +1076,6 @@ The Android Open Source Project Apache Yetus - Audience Annotations Copyright 2015-2017 The Apache Software Foundation -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - Ehcache V3 Copyright 2014-2016 Terracotta, Inc. @@ -1285,19 +1085,9 @@ under the Apache License 2.0 (see: org.ehcache.impl.internal.classes.commonslang Apache Geronimo JCache Spec 1.0 Copyright 2003-2014 The Apache Software Foundation -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - - - - Token provider Copyright 2014-2017 The Apache Software Foundation -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - - Metrics Copyright 2010-2013 Coda Hale and Yammer, Inc. From 06573c566adecb9bd40b641be5b0c3b568f7385f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 11 Nov 2025 17:51:03 -0800 Subject: [PATCH 105/400] [SPARK-54296][PYTHON][TESTS] Optimize the arrow batch slicing tests for cogrouped ### What changes were proposed in this pull request? to optimize newly added tests ### Why are the changes needed? before: ``` Starting test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map (temp output: /__w/spark/spark/python/target/6f28044e-b489-483a-ad37-c4f631875f4e/python3.11__pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map__1r4fgqd_.log) Finished test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map (58s) Starting test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map (temp output: /__w/spark/spark/python/target/93d6495c-37be-4360-bc64-b7f8ccc2020e/python3.11__pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map__phtfyjt1.log) Finished test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map (73s) ``` after: ``` Starting test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map (temp output: /__w/spark/spark/python/target/2240e5cc-4080-477f-866d-cd7183a5de64/python3.11__pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map__msogniqf.log) Finished test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map (28s) Starting test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map (temp output: /__w/spark/spark/python/target/e5d113a7-2779-4d64-88a7-6236fbc6d641/python3.11__pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map__cv8fi_l0.log) Finished test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map (47s) ``` ### Does this PR introduce _any_ user-facing change? No, test-only ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #52991 from zhengruifeng/test_cogroup_opt. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 908b8a324e3470fdaa3889ac6e45b7e54ef41688) Signed-off-by: Dongjoon Hyun --- .../tests/arrow/test_arrow_cogrouped_map.py | 24 +++++++++---------- .../tests/pandas/test_pandas_cogrouped_map.py | 24 +++++++++---------- 2 files changed, 22 insertions(+), 26 deletions(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py b/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py index 2bdd7bda3bc21..6c1c378dcea8e 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py @@ -313,21 +313,19 @@ def arrow_func(key, left, right): self.assertEqual(df2.join(df2).count(), 1) def test_arrow_batch_slicing(self): - df1 = self.spark.range(10000000).select( - (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") - ) + m, n = 100000, 10000 + + df1 = self.spark.range(m).select((sf.col("id") % 2).alias("key"), sf.col("id").alias("v")) cols = {f"col_{i}": sf.col("v") + i for i in range(10)} df1 = df1.withColumns(cols) - df2 = self.spark.range(100000).select( - (sf.col("id") % 4).alias("key"), sf.col("id").alias("v") - ) + df2 = self.spark.range(n).select((sf.col("id") % 4).alias("key"), sf.col("id").alias("v")) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} df2 = df2.withColumns(cols) def summarize(key, left, right): - assert len(left) == 10000000 / 2 or len(left) == 0, len(left) - assert len(right) == 100000 / 4, len(right) + assert len(left) == m / 2 or len(left) == 0, len(left) + assert len(right) == n / 4, len(right) return pa.Table.from_pydict( { "key": [key[0].as_py()], @@ -341,13 +339,13 @@ def summarize(key, left, right): schema = "key long, left_rows long, left_columns long, right_rows long, right_columns long" expected = [ - Row(key=0, left_rows=5000000, left_columns=12, right_rows=25000, right_columns=22), - Row(key=1, left_rows=5000000, left_columns=12, right_rows=25000, right_columns=22), - Row(key=2, left_rows=0, left_columns=12, right_rows=25000, right_columns=22), - Row(key=3, left_rows=0, left_columns=12, right_rows=25000, right_columns=22), + Row(key=0, left_rows=m / 2, left_columns=12, right_rows=n / 4, right_columns=22), + Row(key=1, left_rows=m / 2, left_columns=12, right_rows=n / 4, right_columns=22), + Row(key=2, left_rows=0, left_columns=12, right_rows=n / 4, right_columns=22), + Row(key=3, left_rows=0, left_columns=12, right_rows=n / 4, right_columns=22), ] - for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 1048576), (1000, 1048576)]: + for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py index ab954dd133f3c..8d48bbb794ada 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py @@ -660,21 +660,19 @@ def __test_merge_error( self.__test_merge(left, right, by, fn, output_schema) def test_arrow_batch_slicing(self): - df1 = self.spark.range(10000000).select( - (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") - ) + m, n = 100000, 10000 + + df1 = self.spark.range(m).select((sf.col("id") % 2).alias("key"), sf.col("id").alias("v")) cols = {f"col_{i}": sf.col("v") + i for i in range(10)} df1 = df1.withColumns(cols) - df2 = self.spark.range(100000).select( - (sf.col("id") % 4).alias("key"), sf.col("id").alias("v") - ) + df2 = self.spark.range(n).select((sf.col("id") % 4).alias("key"), sf.col("id").alias("v")) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} df2 = df2.withColumns(cols) def summarize(key, left, right): - assert len(left) == 10000000 / 2 or len(left) == 0, len(left) - assert len(right) == 100000 / 4, len(right) + assert len(left) == m / 2 or len(left) == 0, len(left) + assert len(right) == n / 4, len(right) return pd.DataFrame( { "key": [key[0]], @@ -688,13 +686,13 @@ def summarize(key, left, right): schema = "key long, left_rows long, left_columns long, right_rows long, right_columns long" expected = [ - Row(key=0, left_rows=5000000, left_columns=12, right_rows=25000, right_columns=22), - Row(key=1, left_rows=5000000, left_columns=12, right_rows=25000, right_columns=22), - Row(key=2, left_rows=0, left_columns=12, right_rows=25000, right_columns=22), - Row(key=3, left_rows=0, left_columns=12, right_rows=25000, right_columns=22), + Row(key=0, left_rows=m / 2, left_columns=12, right_rows=n / 4, right_columns=22), + Row(key=1, left_rows=m / 2, left_columns=12, right_rows=n / 4, right_columns=22), + Row(key=2, left_rows=0, left_columns=12, right_rows=n / 4, right_columns=22), + Row(key=3, left_rows=0, left_columns=12, right_rows=n / 4, right_columns=22), ] - for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 1048576), (1000, 1048576)]: + for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { From 8cdd435630495a2b9507c0af841ed96f45f2a545 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 11 Nov 2025 17:53:33 -0800 Subject: [PATCH 106/400] [SPARK-54298][PYTHON][DOCS] Fix doctest of UDFRegistration ### What changes were proposed in this pull request? Fix doctest of UDFRegistration ### Why are the changes needed? to improve test coverage ### Does this PR introduce _any_ user-facing change? yes, doc-only changes ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #52992 from zhengruifeng/doc_test_udf. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 63efbec843bed8baae8efc8ddab6e91e1764aa7b) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/udf.py | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 37aa30cc279f9..c7471d19f7d6f 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -642,25 +642,24 @@ def register( >>> spark.sql("SELECT random_udf()").collect() # doctest: +SKIP [Row(random_udf()=82)] - >>> import pandas as pd # doctest: +SKIP + >>> import pandas as pd >>> from pyspark.sql.functions import pandas_udf - >>> @pandas_udf("integer") # doctest: +SKIP + >>> @pandas_udf("integer") ... def add_one(s: pd.Series) -> pd.Series: ... return s + 1 ... - >>> _ = spark.udf.register("add_one", add_one) # doctest: +SKIP - >>> spark.sql("SELECT add_one(id) FROM range(3)").collect() # doctest: +SKIP + >>> _ = spark.udf.register("add_one", add_one) + >>> spark.sql("SELECT add_one(id) FROM range(3)").collect() [Row(add_one(id)=1), Row(add_one(id)=2), Row(add_one(id)=3)] - >>> @pandas_udf("integer") # doctest: +SKIP + >>> @pandas_udf("integer") ... def sum_udf(v: pd.Series) -> int: ... return v.sum() ... - >>> _ = spark.udf.register("sum_udf", sum_udf) # doctest: +SKIP + >>> _ = spark.udf.register("sum_udf", sum_udf) >>> q = "SELECT sum_udf(v1) FROM VALUES (3, 0), (2, 0), (1, 1) tbl(v1, v2) GROUP BY v2" - >>> spark.sql(q).collect() # doctest: +SKIP + >>> spark.sql(q).sort("sum_udf(v1)").collect() [Row(sum_udf(v1)=1), Row(sum_udf(v1)=5)] - """ # This is to check whether the input function is from a user-defined function or @@ -796,8 +795,13 @@ def _test() -> None: import doctest from pyspark.sql import SparkSession import pyspark.sql.udf + from pyspark.testing.utils import have_pandas, have_pyarrow globs = pyspark.sql.udf.__dict__.copy() + + if not have_pandas or not have_pyarrow: + del pyspark.sql.udf.UDFRegistration.register.__doc__ + spark = SparkSession.builder.master("local[4]").appName("sql.udf tests").getOrCreate() globs["spark"] = spark (failure_count, test_count) = doctest.testmod( From fd134749d276f12c6f6aa2d43b742608094cbd5f Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 11 Nov 2025 17:55:35 -0800 Subject: [PATCH 107/400] [SPARK-54303][SQL] Canonicalize error condition MISSING_CATALOG_ABILITY ### What changes were proposed in this pull request? Rename error condition `_LEGACY_ERROR_TEMP_1184` to `MISSING_CATALOG_ABILITY`, and classify it into multiple sub-classes. ### Why are the changes needed? This is useful for the connect client to classify exceptions via error condition. For example, in https://github.com/apache/spark/pull/52995, it uses `MISSING_CATALOG_ABILITY.VIEWS` to test whether the catalog supports "views" in implementing `getTables`. ### Does this PR introduce _any_ user-facing change? Yes, better error message. ### How was this patch tested? UTs are tuned. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52997 from pan3793/SPARK-54303. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 7ebf5556d9e72ffeee1932ba424872d14e1acbe0) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 58 +++++++++++++++++-- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../catalog/CatalogV2Implicits.scala | 8 +-- .../sql/errors/QueryCompilationErrors.scala | 56 ++++++++++++++++-- .../analysis/ResolveSessionCatalog.scala | 14 ++--- .../connector/DataSourceV2FunctionSuite.scala | 11 ++-- .../sql/connector/DataSourceV2SQLSuite.scala | 8 +-- .../spark/sql/connector/ProcedureSuite.scala | 8 +-- .../command/v2/ShowNamespacesSuite.scala | 14 ++--- 9 files changed, 133 insertions(+), 48 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 4b07c2624d5d5..423189f782580 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4379,6 +4379,59 @@ }, "sqlState" : "XX000" }, + "MISSING_CATALOG_ABILITY" : { + "message" : [ + "Catalog does not support" + ], + "subClass" : { + "CREATE_FUNCTION" : { + "message" : [ + "CREATE FUNCTION." + ] + }, + "DROP_FUNCTION" : { + "message" : [ + "DROP FUNCTION." + ] + }, + "FUNCTIONS" : { + "message" : [ + "functions." + ] + }, + "NAMESPACES" : { + "message" : [ + "namespaces." + ] + }, + "PROCEDURES" : { + "message" : [ + "procedures." + ] + }, + "REFRESH_FUNCTION" : { + "message" : [ + "REFRESH FUNCTION." + ] + }, + "TABLES" : { + "message" : [ + "tables." + ] + }, + "TABLE_VALUED_FUNCTIONS" : { + "message" : [ + "table-valued functions." + ] + }, + "VIEWS" : { + "message" : [ + "views." + ] + } + }, + "sqlState" : "0A000" + }, "MISSING_DATABASE_FOR_V1_SESSION_CATALOG" : { "message" : [ "Database name is not specified in the v1 session catalog. Please ensure to provide a valid database name when interacting with the v1 catalog." @@ -7865,11 +7918,6 @@ "Cannot use \"INTERVAL\" type in the table schema." ] }, - "_LEGACY_ERROR_TEMP_1184" : { - "message" : [ - "Catalog does not support ." - ] - }, "_LEGACY_ERROR_TEMP_1186" : { "message" : [ "Multi-part identifier cannot be empty." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 98c514925fa04..ee589d52a9d0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2172,8 +2172,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor v1SessionCatalog.resolvePersistentTableFunction( ident.asFunctionIdentifier, u.functionArgs) } else { - throw QueryCompilationErrors.missingCatalogAbilityError( - catalog, "table-valued functions") + throw QueryCompilationErrors.missingCatalogTableValuedFunctionsAbilityError( + catalog) } } resolvedFunc.transformAllExpressionsWithPruning( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 282350dda67d3..5cafc42645bec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -105,14 +105,14 @@ private[sql] object CatalogV2Implicits { case tableCatalog: TableCatalog => tableCatalog case _ => - throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "tables") + throw QueryCompilationErrors.missingCatalogTablesAbilityError(plugin) } def asNamespaceCatalog: SupportsNamespaces = plugin match { case namespaceCatalog: SupportsNamespaces => namespaceCatalog case _ => - throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "namespaces") + throw QueryCompilationErrors.missingCatalogNamespacesAbilityError(plugin) } def isFunctionCatalog: Boolean = plugin match { @@ -124,14 +124,14 @@ private[sql] object CatalogV2Implicits { case functionCatalog: FunctionCatalog => functionCatalog case _ => - throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "functions") + throw QueryCompilationErrors.missingCatalogFunctionsAbilityError(plugin) } def asProcedureCatalog: ProcedureCatalog = plugin match { case procedureCatalog: ProcedureCatalog => procedureCatalog case _ => - throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "procedures") + throw QueryCompilationErrors.missingCatalogProceduresAbilityError(plugin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 9847dbd6b197c..246973bbd246b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2206,12 +2206,58 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map.empty) } - def missingCatalogAbilityError(plugin: CatalogPlugin, ability: String): Throwable = { + def missingCatalogFunctionsAbilityError(plugin: CatalogPlugin): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1184", - messageParameters = Map( - "plugin" -> plugin.name, - "ability" -> ability)) + errorClass = "MISSING_CATALOG_ABILITY.FUNCTIONS", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogTableValuedFunctionsAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.TABLE_VALUED_FUNCTIONS", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogCreateFunctionAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.CREATE_FUNCTION", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogDropFunctionAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.DROP_FUNCTION", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogRefreshFunctionAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.REFRESH_FUNCTION", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogNamespacesAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.NAMESPACES", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogProceduresAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.PROCEDURES", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogTablesAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.TABLES", + messageParameters = Map("plugin" -> plugin.name)) + } + + def missingCatalogViewsAbilityError(plugin: CatalogPlugin): Throwable = { + new AnalysisException( + errorClass = "MISSING_CATALOG_ABILITY.VIEWS", + messageParameters = Map("plugin" -> plugin.name)) } def tableValuedArgumentsNotYetImplementedForSqlFunctions( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index b664f52c8c1c5..b1fd0bff1071f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -439,13 +439,13 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) viewSchemaMode = viewSchemaMode) case CreateView(ResolvedIdentifier(catalog, _), _, _, _, _, _, _, _, _, _) => - throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "views") + throw QueryCompilationErrors.missingCatalogViewsAbilityError(catalog) case ShowViews(ns: ResolvedNamespace, pattern, output) => ns match { case ResolvedDatabaseInSessionCatalog(db) => ShowViewsCommand(db, pattern, output) case _ => - throw QueryCompilationErrors.missingCatalogAbilityError(ns.catalog, "views") + throw QueryCompilationErrors.missingCatalogViewsAbilityError(ns.catalog) } // If target is view, force use v1 command @@ -463,7 +463,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if (isSessionCatalog(catalog)) { DescribeFunctionCommand(func.asInstanceOf[V1Function].info, extended) } else { - throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "functions") + throw QueryCompilationErrors.missingCatalogFunctionsAbilityError(catalog) } case ShowFunctions( @@ -476,7 +476,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) identifier.asFunctionIdentifier) DropFunctionCommand(funcIdentifier, ifExists, false) } else { - throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "DROP FUNCTION") + throw QueryCompilationErrors.missingCatalogDropFunctionAbilityError(catalog) } case RefreshFunction(ResolvedPersistentFunc(catalog, identifier, _)) => @@ -485,7 +485,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) identifier.asFunctionIdentifier) RefreshFunctionCommand(funcIdentifier.database, funcIdentifier.funcName) } else { - throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "REFRESH FUNCTION") + throw QueryCompilationErrors.missingCatalogRefreshFunctionAbilityError(catalog) } case CreateFunction( @@ -499,7 +499,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) replace) case CreateFunction(ResolvedIdentifier(catalog, _), _, _, _, _) => - throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "CREATE FUNCTION") + throw QueryCompilationErrors.missingCatalogCreateFunctionAbilityError(catalog) case c @ CreateUserDefinedFunction( ResolvedIdentifierInSessionCatalog(ident), _, _, _, _, _, _, _, _, _, _, _) => @@ -520,7 +520,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case CreateUserDefinedFunction( ResolvedIdentifier(catalog, _), _, _, _, _, _, _, _, _, _, _, _) => - throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "CREATE FUNCTION") + throw QueryCompilationErrors.missingCatalogCreateFunctionAbilityError(catalog) } private def constructV1TableCmd( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala index 6b0fd6084099c..c6f2da686fe96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala @@ -161,8 +161,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException]( sql("SELECT testcat.strlen('abc')").collect() ), - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map("plugin" -> "testcat", "ability" -> "functions") + condition = "MISSING_CATALOG_ABILITY.FUNCTIONS", + parameters = Map("plugin" -> "testcat") ) } } @@ -174,11 +174,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException] { sql("DESCRIBE FUNCTION testcat.abc") }, - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map( - "plugin" -> "testcat", - "ability" -> "functions" - ) + condition = "MISSING_CATALOG_ABILITY.FUNCTIONS", + parameters = Map("plugin" -> "testcat") ) checkError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 21538ec8e44ae..bf10d50db1423 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1322,8 +1322,8 @@ class DataSourceV2SQLSuiteV1Filter test("ShowViews: using v2 catalog, command not supported.") { checkError( exception = analysisException("SHOW VIEWS FROM testcat"), - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map("plugin" -> "testcat", "ability" -> "views")) + condition = "MISSING_CATALOG_ABILITY.VIEWS", + parameters = Map("plugin" -> "testcat")) } test("create/replace/alter table - reserved properties") { @@ -2517,8 +2517,8 @@ class DataSourceV2SQLSuiteV1Filter val v = "testcat.ns1.ns2.v" checkError( exception = analysisException(s"CREATE VIEW $v AS SELECT 1"), - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map("plugin" -> "testcat", "ability" -> "views")) + condition = "MISSING_CATALOG_ABILITY.VIEWS", + parameters = Map("plugin" -> "testcat")) } test("global temp view should not be masked by v2 catalog") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/ProcedureSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/ProcedureSuite.scala index 8a6fcae198513..a88e6dbbe6dee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/ProcedureSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/ProcedureSuite.scala @@ -176,16 +176,16 @@ class ProcedureSuite extends QueryTest with SharedSparkSession with BeforeAndAft exception = intercept[AnalysisException]( sql("CALL testcat.procedure(1, 2)") ), - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map("plugin" -> "testcat", "ability" -> "procedures") + condition = "MISSING_CATALOG_ABILITY.PROCEDURES", + parameters = Map("plugin" -> "testcat") ) checkError( exception = intercept[AnalysisException]( sql("SHOW PROCEDURES IN testcat") ), - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map("plugin" -> "testcat", "ability" -> "procedures") + condition = "MISSING_CATALOG_ABILITY.PROCEDURES", + parameters = Map("plugin" -> "testcat") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala index 44a1bcad46a03..1a84a97a6230e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala @@ -44,11 +44,8 @@ class ShowNamespacesSuite extends command.ShowNamespacesSuiteBase with CommandSu exception = intercept[AnalysisException] { sql("SHOW NAMESPACES") }, - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map( - "plugin" -> "testcat_no_namespace", - "ability" -> "namespaces" - ) + condition = "MISSING_CATALOG_ABILITY.NAMESPACES", + parameters = Map("plugin" -> "testcat_no_namespace") ) } } @@ -58,11 +55,8 @@ class ShowNamespacesSuite extends command.ShowNamespacesSuiteBase with CommandSu exception = intercept[AnalysisException] { sql("SHOW NAMESPACES in testcat_no_namespace") }, - condition = "_LEGACY_ERROR_TEMP_1184", - parameters = Map( - "plugin" -> "testcat_no_namespace", - "ability" -> "namespaces" - ) + condition = "MISSING_CATALOG_ABILITY.NAMESPACES", + parameters = Map("plugin" -> "testcat_no_namespace") ) } } From b57ed379041425dde30508d315d18cfe38b9c49e Mon Sep 17 00:00:00 2001 From: Yicong-Huang <17627829+Yicong-Huang@users.noreply.github.com> Date: Tue, 11 Nov 2025 17:57:23 -0800 Subject: [PATCH 108/400] [SPARK-54297][PYTHON][SQL][TESTS] Optimize tests for iterator of dataframe API in `applyInPandas` ### What changes were proposed in this pull request? We use these tests to ensure that large datasets are processed exclusively through the new iterator API and that loading them entirely into memory would fail. However, the original tests were too slow and resource-intensive. This PR optimizes the test to achieve the same validation goal while using significantly fewer resources. ### Why are the changes needed? To save time and resources for CI. Before: ``` 7d04fd12050e/python3.11__pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map__6iwxrtx4.log) Finished test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map **(291s)** Starting test(python3.11): pyspark.sql.tests.pandas.test_pandas_grouped_map (temp output: /__w/spark/spark/python/target/379cea6f-7421-4f81-ae7b-1c22baddaa92/python3.11__pyspark.sql.tests.pandas.test_pandas_grouped_map__km0800fd.log) Finished test(python3.11): pyspark.sql.tests.pandas.test_pandas_grouped_map **(300s)** ``` After: ``` Starting test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map (temp output: /__w/spark/spark/python/target/4cbd5d5f-18b9-48ba-8bf4-30fa18698355/python3.11__pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map__4o3yj9ck.log) Finished test(python3.11): pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map (130s) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? CI test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53002 from Yicong-Huang/SPARK-54297/fix/optimize-test. Authored-by: Yicong-Huang <17627829+Yicong-Huang@users.noreply.github.com> Signed-off-by: Dongjoon Hyun (cherry picked from commit f3f1449a9aac6d87c235f710bdbf9ffbbd715b9c) Signed-off-by: Dongjoon Hyun --- .../tests/pandas/test_pandas_grouped_map.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index a2f31bacf8127..79895bb1a01b3 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -938,14 +938,14 @@ def test(pdf): self.assertEqual(row[1], 123) def test_arrow_batch_slicing(self): - df = self.spark.range(10000000).select( + df = self.spark.range(100000).select( (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") ) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} df = df.withColumns(cols) def min_max_v(pdf): - assert len(pdf) == 10000000 / 2, len(pdf) + assert len(pdf) == 100000 / 2, len(pdf) return pd.DataFrame( { "key": [pdf.key.iloc[0]], @@ -958,7 +958,7 @@ def min_max_v(pdf): df.groupby("key").agg(sf.min("v").alias("min"), sf.max("v").alias("max")).sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 1048576), (1000, 1048576)]: + for maxRecords, maxBytes in [(1000, 4096), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { @@ -1057,7 +1057,7 @@ def sum_func( self.assertEqual(result[1][1], 18.0) def test_apply_in_pandas_iterator_batch_slicing(self): - df = self.spark.range(10000000).select( + df = self.spark.range(100000).select( (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") ) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} @@ -1073,7 +1073,7 @@ def min_max_v(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: key_val = batch.key.iloc[0] combined = pd.concat(all_data, ignore_index=True) - assert len(combined) == 10000000 / 2, len(combined) + assert len(combined) == 100000 / 2, len(combined) yield pd.DataFrame( { @@ -1092,7 +1092,7 @@ def min_max_v(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: .sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 1048576), (1000, 1048576)]: + for maxRecords, maxBytes in [(1000, 4096), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { @@ -1109,7 +1109,7 @@ def min_max_v(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: self.assertEqual(expected, result) def test_apply_in_pandas_iterator_with_keys_batch_slicing(self): - df = self.spark.range(10000000).select( + df = self.spark.range(100000).select( (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") ) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} @@ -1124,7 +1124,7 @@ def min_max_v( all_data.append(batch) combined = pd.concat(all_data, ignore_index=True) - assert len(combined) == 10000000 / 2, len(combined) + assert len(combined) == 100000 / 2, len(combined) yield pd.DataFrame( { @@ -1138,7 +1138,7 @@ def min_max_v( df.groupby("key").agg(sf.min("v").alias("min"), sf.max("v").alias("max")).sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 1048576), (1000, 1048576)]: + for maxRecords, maxBytes in [(1000, 4096), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { From 715fe5fcdc99218319847704981258f97ada6a8f Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 11 Nov 2025 18:10:37 -0800 Subject: [PATCH 109/400] [SPARK-52439][SQL] Support check constraint with null value ### What changes were proposed in this pull request? Support check constraint with null value by adding `ImplicitCastInputTypes` trait to `CheckConstraint`. ### Why are the changes needed? Otherwise there will be an error from `V2ExpressionBuilder` since the constraint expression is of `NullType` instead of `BooleanType`. ### Does this PR introduce _any_ user-facing change? No, this is not released yet. ### How was this patch tested? New UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #53004 from gengliangwang/nullCheckConstraint. Authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun (cherry picked from commit 03a0e05acfcff5b38d5d3fa9b495fc207a9600e5) Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/constraints.scala | 7 ++- .../command/v2/CheckConstraintSuite.scala | 44 +++++++++++++++++++ 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index a27460e2be1cd..1a7e3b03c0e6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder import org.apache.spark.sql.connector.catalog.constraints.Constraint import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType} trait TableConstraint extends Expression with Unevaluable { /** Convert to a data source v2 constraint */ @@ -122,9 +122,12 @@ case class CheckConstraint( override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) extends UnaryExpression - with TableConstraint { + with TableConstraint + with ImplicitCastInputTypes { // scalastyle:on line.size.limit + override def inputTypes: Seq[AbstractDataType] = Seq(BooleanType) + def toV2Constraint: Constraint = { val predicate = new V2ExpressionBuilder(child, true).buildPredicate().orNull val enforced = userProvidedCharacteristic.enforced.getOrElse(true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index de03653fc9165..1d14d710744a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.catalog.constraints.Check import org.apache.spark.sql.execution.command.DDLCommandTestUtils @@ -1150,4 +1151,47 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } } } + + test("Check constraint with constant valid expression should be optimized out") { + Seq( + "1 > 0", + "abs(-99) < 100", + "null", + "current_date() > DATE'2023-01-01'" + ).foreach { constant => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + sql(s"CREATE TABLE $t (id INT, value INT," + + s" CONSTRAINT positive_id CHECK ($constant)) $defaultUsing") + val optimizedPlan = + sql(s"INSERT INTO $t VALUES (1, 10), (2, 20)").queryExecution.optimizedPlan + val filter = optimizedPlan.collectFirst { + case f: Filter => f + } + assert(filter.isEmpty) + } + } + } + + test("Check constraint with constant invalid expression should throw error") { + Seq( + "1 < 0", + "abs(-99) > 100", + "current_date() < DATE'2023-01-01'" + ).foreach { constant => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + sql(s"CREATE TABLE $t (id INT, value INT," + + s" CONSTRAINT positive_id CHECK ($constant)) $defaultUsing") + val error = intercept[SparkRuntimeException] { + sql(s"INSERT INTO $t VALUES (1, 10), (2, 20)") + } + checkError( + exception = error, + condition = "CHECK_CONSTRAINT_VIOLATION", + sqlState = "23001", + parameters = Map("constraintName" -> "positive_id", "expression" -> constant, + "values" -> "") + ) + } + } + } } From 42392202eee39d2ab6fad387f71fd56294032aaa Mon Sep 17 00:00:00 2001 From: cty123 Date: Wed, 12 Nov 2025 10:33:03 +0800 Subject: [PATCH 110/400] [SPARK-54270][CONNECT] SparkConnectResultSet get* methods should call checkOpen and check index boundary ### What changes were proposed in this pull request? This PR aims to do a minor correction on the current get* functions from `SparkConnectResultSet` class. As previously discussed in the PR https://github.com/apache/spark/pull/52947, >1. For every getter function, if the statement is closed, the ResultSet should be unusable. I have verified this with MySQL driver and Postgresql driver. > >2. Right now when index goes out of bound, it throws `java.lang.ArrayIndexOutOfBoundsException`, but based on the specification on `java.sql.ResultSet` which is implemented by `SparkConnectResultSet` class, it should throw `java.sql.SQLException` > >``` > * throws SQLException if the columnIndex is not valid; >``` This PR proposes a unified wrapper function called `getColumnValue(columnIndex: Int)` that aims to wrap the `checkOpen` as well as the index boundary check. ### Why are the changes needed? Currently the get* functions don't follow the expected behaviors of `java.sql.ResultSet`. It's technically not a big problem, but since the `SparkConnectResultSet` aims to implement the `java.sql.ResultSet` class, it should strictly follow the specification documented on the interface definition. ### Does this PR introduce _any_ user-facing change? This PR is a small fix related to a new feature introduced recently. ### How was this patch tested? I added 2 tests each covering a bullet point I named above. These 2 functions calls all the get* functions inside `SparkConnectResultSet` class to make sure the correct exception(java.sql.SQLException) is thrown. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52988 from cty123/cty123/address-spark-connect-getters. Lead-authored-by: cty123 Co-authored-by: cty Signed-off-by: yangjie01 (cherry picked from commit c21d5a45073f0509d9d83219dd36e40ccc3868c8) Signed-off-by: yangjie01 --- .../client/jdbc/SparkConnectResultSet.scala | 88 +++++++------------ .../jdbc/SparkConnectJdbcDataTypeSuite.scala | 55 +++++++++++- 2 files changed, 84 insertions(+), 59 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index 23c2315400fff..ff02cd73dcc61 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -76,6 +76,24 @@ class SparkConnectResultSet( } } + private def getColumnValue[T](columnIndex: Int, defaultVal: T)(getter: Int => T): T = { + checkOpen() + // the passed index value is 1-indexed, but the underlying array is 0-indexed + val index = columnIndex - 1 + if (index < 0 || index >= currentRow.length) { + throw new SQLException(s"The column index is out of range: $columnIndex, " + + s"number of columns: ${currentRow.length}.") + } + + if (currentRow.isNullAt(index)) { + _wasNull = true + defaultVal + } else { + _wasNull = false + getter(index) + } + } + override def findColumn(columnLabel: String): Int = { sparkResult.schema.getFieldIndex(columnLabel) match { case Some(i) => i + 1 @@ -85,75 +103,35 @@ class SparkConnectResultSet( } override def getString(columnIndex: Int): String = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return null - } - _wasNull = false - String.valueOf(currentRow.get(columnIndex - 1)) + getColumnValue(columnIndex, null: String) { idx => String.valueOf(currentRow.get(idx)) } } override def getBoolean(columnIndex: Int): Boolean = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return false - } - _wasNull = false - currentRow.getBoolean(columnIndex - 1) + getColumnValue(columnIndex, false) { idx => currentRow.getBoolean(idx) } } override def getByte(columnIndex: Int): Byte = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return 0.toByte - } - _wasNull = false - currentRow.getByte(columnIndex - 1) + getColumnValue(columnIndex, 0.toByte) { idx => currentRow.getByte(idx) } } override def getShort(columnIndex: Int): Short = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return 0.toShort - } - _wasNull = false - currentRow.getShort(columnIndex - 1) + getColumnValue(columnIndex, 0.toShort) { idx => currentRow.getShort(idx) } } override def getInt(columnIndex: Int): Int = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return 0 - } - _wasNull = false - currentRow.getInt(columnIndex - 1) + getColumnValue(columnIndex, 0) { idx => currentRow.getInt(idx) } } override def getLong(columnIndex: Int): Long = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return 0L - } - _wasNull = false - currentRow.getLong(columnIndex - 1) + getColumnValue(columnIndex, 0.toLong) { idx => currentRow.getLong(idx) } } override def getFloat(columnIndex: Int): Float = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return 0.toFloat - } - _wasNull = false - currentRow.getFloat(columnIndex - 1) + getColumnValue(columnIndex, 0.toFloat) { idx => currentRow.getFloat(idx) } } override def getDouble(columnIndex: Int): Double = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return 0.toDouble - } - _wasNull = false - currentRow.getDouble(columnIndex - 1) + getColumnValue(columnIndex, 0.toDouble) { idx => currentRow.getDouble(idx) } } override def getBigDecimal(columnIndex: Int, scale: Int): java.math.BigDecimal = @@ -240,12 +218,9 @@ class SparkConnectResultSet( } override def getObject(columnIndex: Int): AnyRef = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return null + getColumnValue(columnIndex, null: AnyRef) { idx => + currentRow.get(idx).asInstanceOf[AnyRef] } - _wasNull = false - currentRow.get(columnIndex - 1).asInstanceOf[AnyRef] } override def getObject(columnLabel: String): AnyRef = @@ -258,12 +233,9 @@ class SparkConnectResultSet( throw new SQLFeatureNotSupportedException override def getBigDecimal(columnIndex: Int): java.math.BigDecimal = { - if (currentRow.isNullAt(columnIndex - 1)) { - _wasNull = true - return null + getColumnValue(columnIndex, null: java.math.BigDecimal) { idx => + currentRow.getDecimal(idx) } - _wasNull = false - currentRow.getDecimal(columnIndex - 1) } override def getBigDecimal(columnLabel: String): java.math.BigDecimal = diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index 089c1d7fdf0d4..217142287b130 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.connect.client.jdbc -import java.sql.Types +import java.sql.{ResultSet, SQLException, Types} import org.apache.spark.sql.connect.client.jdbc.test.JdbcHelper import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession} @@ -248,4 +248,57 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess } } } + + test("getter functions column index out of bound") { + Seq( + ("'foo'", (rs: ResultSet) => rs.getString(999)), + ("true", (rs: ResultSet) => rs.getBoolean(999)), + ("cast(1 AS BYTE)", (rs: ResultSet) => rs.getByte(999)), + ("cast(1 AS SHORT)", (rs: ResultSet) => rs.getShort(999)), + ("cast(1 AS INT)", (rs: ResultSet) => rs.getInt(999)), + ("cast(1 AS BIGINT)", (rs: ResultSet) => rs.getLong(999)), + ("cast(1 AS FLOAT)", (rs: ResultSet) => rs.getFloat(999)), + ("cast(1 AS DOUBLE)", (rs: ResultSet) => rs.getDouble(999)), + ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(999)) + ).foreach { + case (query, getter) => + withExecuteQuery(s"SELECT $query") { rs => + assert(rs.next()) + val exception = intercept[SQLException] { + getter(rs) + } + assert(exception.getMessage() === + "The column index is out of range: 999, number of columns: 1.") + } + } + } + + test("getter functions called after statement closed") { + Seq( + ("'foo'", (rs: ResultSet) => rs.getString(1), "foo"), + ("true", (rs: ResultSet) => rs.getBoolean(1), true), + ("cast(1 AS BYTE)", (rs: ResultSet) => rs.getByte(1), 1.toByte), + ("cast(1 AS SHORT)", (rs: ResultSet) => rs.getShort(1), 1.toShort), + ("cast(1 AS INT)", (rs: ResultSet) => rs.getInt(1), 1.toInt), + ("cast(1 AS BIGINT)", (rs: ResultSet) => rs.getLong(1), 1.toLong), + ("cast(1 AS FLOAT)", (rs: ResultSet) => rs.getFloat(1), 1.toFloat), + ("cast(1 AS DOUBLE)", (rs: ResultSet) => rs.getDouble(1), 1.toDouble), + ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(1), + new java.math.BigDecimal("1.00000")) + ).foreach { + case (query, getter, expectedValue) => + var resultSet: Option[ResultSet] = None + withExecuteQuery(s"SELECT $query") { rs => + assert(rs.next()) + assert(getter(rs) === expectedValue) + assert(!rs.wasNull) + resultSet = Some(rs) + } + assert(resultSet.isDefined) + val exception = intercept[SQLException] { + getter(resultSet.get) + } + assert(exception.getMessage() === "JDBC Statement is closed.") + } + } } From 1708acb79d03cf5b26c2b5227b0fabed701b0ff0 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 11 Nov 2025 21:26:51 -0800 Subject: [PATCH 111/400] [SPARK-54113][CONNECT] Support getTables for SparkConnectDatabaseMetaData ### What changes were proposed in this pull request? Implement `getTableTypes` and `getTables` methods defined in `java.sql.DatabaseMetaData` for `SparkConnectDatabaseMetaData`. ```java /** * Retrieves the table types available in this database. The results * are ordered by table type. * *

The table type is: *

    *
  1. TABLE_TYPE String {code =>} table type. Typical types are "TABLE", * "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY", * "LOCAL TEMPORARY", "ALIAS", "SYNONYM". *
* * return a {code ResultSet} object in which each row has a * single {code String} column that is a table type * throws SQLException if a database access error occurs */ ResultSet getTableTypes() throws SQLException; /** * Retrieves a description of the tables available in the given catalog. * Only table descriptions matching the catalog, schema, table * name and type criteria are returned. They are ordered by * {code TABLE_TYPE}, {code TABLE_CAT}, * {code TABLE_SCHEM} and {code TABLE_NAME}. *

* Each table description has the following columns: *

    *
  1. TABLE_CAT String {code =>} table catalog (may be {code null}) *
  2. TABLE_SCHEM String {code =>} table schema (may be {code null}) *
  3. TABLE_NAME String {code =>} table name *
  4. TABLE_TYPE String {code =>} table type. Typical types are "TABLE", * "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY", * "LOCAL TEMPORARY", "ALIAS", "SYNONYM". *
  5. REMARKS String {code =>} explanatory comment on the table (may be {code null}) *
  6. TYPE_CAT String {code =>} the types catalog (may be {code null}) *
  7. TYPE_SCHEM String {code =>} the types schema (may be {code null}) *
  8. TYPE_NAME String {code =>} type name (may be {code null}) *
  9. SELF_REFERENCING_COL_NAME String {code =>} name of the designated * "identifier" column of a typed table (may be {code null}) *
  10. REF_GENERATION String {code =>} specifies how values in * SELF_REFERENCING_COL_NAME are created. Values are * "SYSTEM", "USER", "DERIVED". (may be {code null}) *
* *

Note: Some databases may not return information for * all tables. * * param catalog a catalog name; must match the catalog name as it * is stored in the database; "" retrieves those without a catalog; * {code null} means that the catalog name should not be used to narrow * the search * param schemaPattern a schema name pattern; must match the schema name * as it is stored in the database; "" retrieves those without a schema; * {code null} means that the schema name should not be used to narrow * the search * param tableNamePattern a table name pattern; must match the * table name as it is stored in the database * param types a list of table types, which must be from the list of table types * returned from {link #getTableTypes},to include; {code null} returns * all types * return {code ResultSet} - each row is a table description * throws SQLException if a database access error occurs * see #getSearchStringEscape */ ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String types[]) throws SQLException; ``` ### Why are the changes needed? Enhance API coverage of the Connect JDBC driver, for example, `get[Catalogs|Schemas|Tables|...]` APIs are used by SQL GUI tools such as DBeaver for displaying the tree category. ### Does this PR introduce _any_ user-facing change? No, Connect JDBC driver is a new feature under development. ### How was this patch tested? New UTs are added. Tested via DBeaver - the catalog/schema/table tree works now. image ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52995 from pan3793/SPARK-54113. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit a3a8d320581af2384606377198b2f734255ca520) Signed-off-by: Dongjoon Hyun --- .../jdbc/SparkConnectDatabaseMetaData.scala | 127 ++++++++++- .../SparkConnectDatabaseMetaDataSuite.scala | 204 ++++++++++++++++++ 2 files changed, 321 insertions(+), 10 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala index 13dd4d57662b6..47dfef2eb7058 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.connect.client.jdbc import java.sql.{Array => _, _} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} +import org.apache.spark.SparkThrowable import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.util.QuotingUtils._ import org.apache.spark.sql.connect @@ -315,12 +316,12 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas // |-- TABLE_SCHEM: string (nullable = false) // |-- TABLE_CATALOG: string (nullable = false) private def getSchemasDataFrame( - catalog: String, schemaPattern: String): connect.DataFrame = { + catalog: String, schemaPatternOpt: Option[String]): connect.DataFrame = { - val schemaFilterExpr = if (isNullOrWildcard(schemaPattern)) { - lit(true) - } else { - $"TABLE_SCHEM".like(schemaPattern) + val schemaFilterExpr = schemaPatternOpt match { + case None => $"TABLE_SCHEM".equalTo(conn.spark.catalog.currentDatabase) + case Some(schemaPattern) if isNullOrWildcard(schemaPattern) => lit(true) + case Some(schemaPattern) => $"TABLE_SCHEM".like(schemaPattern) } def internalGetSchemas( @@ -355,20 +356,124 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas override def getSchemas(catalog: String, schemaPattern: String): ResultSet = { conn.checkOpen() - val df = getSchemasDataFrame(catalog, schemaPattern) + val df = getSchemasDataFrame(catalog, Some(schemaPattern)) .orderBy("TABLE_CATALOG", "TABLE_SCHEM") new SparkConnectResultSet(df.collectResult()) } - override def getTableTypes: ResultSet = - throw new SQLFeatureNotSupportedException + override def getTableTypes: ResultSet = { + conn.checkOpen() + + val df = TABLE_TYPES.toDF("TABLE_TYPE") + .orderBy("TABLE_TYPE") + new SparkConnectResultSet(df.collectResult()) + } + + // Schema of the returned DataFrame is: + // |-- TABLE_CAT: string (nullable = false) + // |-- TABLE_SCHEM: string (nullable = false) + // |-- TABLE_NAME: string (nullable = false) + // |-- TABLE_TYPE: string (nullable = false) + // |-- REMARKS: string (nullable = false) + // |-- TYPE_CAT: string (nullable = false) + // |-- TYPE_SCHEM: string (nullable = false) + // |-- TYPE_NAME: string (nullable = false) + // |-- SELF_REFERENCING_COL_NAME: string (nullable = false) + // |-- REF_GENERATION: string (nullable = false) + private def getTablesDataFrame( + catalog: String, + schemaPattern: String, + tableNamePattern: String): connect.DataFrame = { + + val catalogSchemasDf = if (schemaPattern == "") { + getSchemasDataFrame(catalog, None) + } else { + getSchemasDataFrame(catalog, Some(schemaPattern)) + } + + val catalogSchemas = catalogSchemasDf.collect() + .map { row => (row.getString(1), row.getString(0)) } + + val tableNameFilterExpr = if (isNullOrWildcard(tableNamePattern)) { + lit(true) + } else { + $"TABLE_NAME".like(tableNamePattern) + } + + val emptyDf = conn.spark.emptyDataFrame + .withColumn("TABLE_CAT", lit("")) + .withColumn("TABLE_SCHEM", lit("")) + .withColumn("TABLE_NAME", lit("")) + .withColumn("TABLE_TYPE", lit("")) + .withColumn("REMARKS", lit("")) + .withColumn("TYPE_CAT", lit("")) + .withColumn("TYPE_SCHEM", lit("")) + .withColumn("TYPE_NAME", lit("")) + .withColumn("SELF_REFERENCING_COL_NAME", lit("")) + .withColumn("REF_GENERATION", lit("")) + + catalogSchemas.map { case (catalog, schema) => + val viewDf = try { + conn.spark + .sql(s"SHOW VIEWS IN ${quoteNameParts(Seq(catalog, schema))}") + .select($"namespace".as("TABLE_SCHEM"), $"viewName".as("TABLE_NAME")) + .filter(tableNameFilterExpr) + } catch { + case st: SparkThrowable if st.getCondition == "MISSING_CATALOG_ABILITY.VIEWS" => + emptyDf.select("TABLE_SCHEM", "TABLE_NAME") + } + + val tableDf = try { + conn.spark + .sql(s"SHOW TABLES IN ${quoteNameParts(Seq(catalog, schema))}") + .select($"namespace".as("TABLE_SCHEM"), $"tableName".as("TABLE_NAME")) + .filter(tableNameFilterExpr) + .exceptAll(viewDf) + } catch { + case st: SparkThrowable if st.getCondition == "MISSING_CATALOG_ABILITY.TABLES" => + emptyDf.select("TABLE_SCHEM", "TABLE_NAME") + } + + tableDf.withColumn("TABLE_TYPE", lit("TABLE")) + .unionAll(viewDf.withColumn("TABLE_TYPE", lit("VIEW"))) + .withColumn("TABLE_CAT", lit(catalog)) + .withColumn("REMARKS", lit("")) + .withColumn("TYPE_CAT", lit("")) + .withColumn("TYPE_SCHEM", lit("")) + .withColumn("TYPE_NAME", lit("")) + .withColumn("SELF_REFERENCING_COL_NAME", lit("")) + .withColumn("REF_GENERATION", lit("")) + .select("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE", "REMARKS", + "TYPE_CAT", "TYPE_SCHEM", "TYPE_NAME", "SELF_REFERENCING_COL_NAME", + "REF_GENERATION") + }.fold(emptyDf) { (l, r) => l.unionAll(r) } + } override def getTables( catalog: String, schemaPattern: String, tableNamePattern: String, - types: Array[String]): ResultSet = - throw new SQLFeatureNotSupportedException + types: Array[String]): ResultSet = { + conn.checkOpen() + + if (types != null) { + val unsupported = types.diff(TABLE_TYPES) + if (unsupported.nonEmpty) { + throw new SQLException( + "The requested table types contains unsupported items: " + + s"${unsupported.mkString(", ")}. Available table types are: " + + s"${TABLE_TYPES.mkString(", ")}.") + } + } + + var df = getTablesDataFrame(catalog, schemaPattern, tableNamePattern) + .orderBy("TABLE_TYPE", "TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME") + + if (types != null) { + df = df.filter($"TABLE_TYPE".isInCollection(types)) + } + new SparkConnectResultSet(df.collectResult()) + } override def getColumns( catalog: String, @@ -618,4 +723,6 @@ object SparkConnectDatabaseMetaData { "XMLFOREST", "XMLNAMESPACES", "XMLPARSE", "XMLPI", "XMLROOT", "XMLSERIALIZE", "YEAR" ) + + private[jdbc] val TABLE_TYPES = Seq("TABLE", "VIEW") } diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala index c3d891bc38c38..3b6622f97a3e4 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala @@ -343,4 +343,208 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark } } } + + test("SparkConnectDatabaseMetaData getTableTypes") { + withConnection { conn => + val metadata = conn.getMetaData + Using.resource(metadata.getTableTypes) { rs => + val types = new Iterator[String] { + def hasNext: Boolean = rs.next() + def next(): String = rs.getString("TABLE_TYPE") + }.toSeq + // results are ordered by TABLE_TYPE + assert(types === Seq("TABLE", "VIEW")) + } + } + } + + test("SparkConnectDatabaseMetaData getTables") { + + case class GetTableResult( + TABLE_CAT: String, + TABLE_SCHEM: String, + TABLE_NAME: String, + TABLE_TYPE: String, + REMARKS: String, + TYPE_CAT: String, + TYPE_SCHEM: String, + TYPE_NAME: String, + SELF_REFERENCING_COL_NAME: String, + REF_GENERATION: String) + + def verifyEmptyStringFields(result: GetTableResult): Unit = { + assert(result.REMARKS === "") + assert(result.TYPE_CAT === "") + assert(result.TYPE_SCHEM === "") + assert(result.TYPE_NAME === "") + assert(result.SELF_REFERENCING_COL_NAME === "") + assert(result.REF_GENERATION === "") + } + + def verifyGetTables( + getTables: () => ResultSet)(verify: Seq[GetTableResult] => Unit): Unit = { + Using.resource(getTables()) { rs => + val getTableResults = new Iterator[GetTableResult] { + def hasNext: Boolean = rs.next() + def next(): GetTableResult = GetTableResult( + TABLE_CAT = rs.getString("TABLE_CAT"), + TABLE_SCHEM = rs.getString("TABLE_SCHEM"), + TABLE_NAME = rs.getString("TABLE_NAME"), + TABLE_TYPE = rs.getString("TABLE_TYPE"), + REMARKS = rs.getString("REMARKS"), + TYPE_CAT = rs.getString("TYPE_CAT"), + TYPE_SCHEM = rs.getString("TYPE_SCHEM"), + TYPE_NAME = rs.getString("TYPE_NAME"), + SELF_REFERENCING_COL_NAME = rs.getString("SELF_REFERENCING_COL_NAME"), + REF_GENERATION = rs.getString("REF_GENERATION")) + }.toSeq + verify(getTableResults) + } + } + + withConnection { conn => + implicit val spark: SparkSession = conn.asInstanceOf[SparkConnectConnection].spark + + // this catalog does not support view + registerCatalog("testcat", TEST_IN_MEMORY_CATALOG) + + spark.sql("CREATE DATABASE IF NOT EXISTS testcat.t_db1") + spark.sql("CREATE TABLE IF NOT EXISTS testcat.t_db1.t_t1 (id INT)") + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db1") + spark.sql("CREATE TABLE IF NOT EXISTS spark_catalog.db1.t1 (id INT)") + spark.sql("CREATE TABLE IF NOT EXISTS spark_catalog.db1.t_2 (id INT)") + spark.sql( + """CREATE VIEW IF NOT EXISTS spark_catalog.db1.t1_v AS + |SELECT id FROM spark_catalog.db1.t1 + |""".stripMargin) + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db_2") + spark.sql("CREATE TABLE IF NOT EXISTS spark_catalog.db_2.t_2 (id INT)") + spark.sql( + """CREATE VIEW IF NOT EXISTS spark_catalog.db_2.t_2_v AS + |SELECT id FROM spark_catalog.db_2.t_2 + |""".stripMargin) + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db_") + spark.sql("CREATE TABLE IF NOT EXISTS spark_catalog.db_.t_ (id INT)") + + val metadata = conn.getMetaData + + // no need to care about "testcat" because it is memory based and session isolated, + // also is inaccessible from another SparkSession + withDatabase("spark_catalog.db1", "spark_catalog.db_2", "spark_catalog.db_") { + // list tables in all catalogs and schemas + val getTablesInAllCatalogsAndSchemas = List(null, "%").flatMap { database => + List(null, "%").flatMap { table => + List(null, Array("TABLE", "VIEW")).map { tableTypes => + () => metadata.getTables(null, database, table, tableTypes) + } + } + } + + getTablesInAllCatalogsAndSchemas.foreach { getTables => + verifyGetTables(getTables) { getTableResults => + // results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM and TABLE_NAME + assert { + getTableResults.map { result => + (result.TABLE_TYPE, result.TABLE_CAT, result.TABLE_SCHEM, result.TABLE_NAME) + } === Seq( + ("TABLE", "spark_catalog", "db1", "t1"), + ("TABLE", "spark_catalog", "db1", "t_2"), + ("TABLE", "spark_catalog", "db_", "t_"), + ("TABLE", "spark_catalog", "db_2", "t_2"), + ("TABLE", "testcat", "t_db1", "t_t1"), + ("VIEW", "spark_catalog", "db1", "t1_v"), + ("VIEW", "spark_catalog", "db_2", "t_2_v")) + } + getTableResults.foreach(verifyEmptyStringFields) + } + } + + // list tables with table types + val se = intercept[SQLException] { + metadata.getTables("spark_catalog", "foo", "bar", Array("TABLE", "MATERIALIZED VIEW")) + } + assert(se.getMessage === + "The requested table types contains unsupported items: MATERIALIZED VIEW. " + + "Available table types are: TABLE, VIEW.") + + verifyGetTables { + () => metadata.getTables("spark_catalog", "db1", "%", Array("TABLE")) + } { getTableResults => + // results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM and TABLE_NAME + assert { + getTableResults.map { result => + (result.TABLE_TYPE, result.TABLE_CAT, result.TABLE_SCHEM, result.TABLE_NAME) + } === Seq( + ("TABLE", "spark_catalog", "db1", "t1"), + ("TABLE", "spark_catalog", "db1", "t_2")) + } + getTableResults.foreach(verifyEmptyStringFields) + } + + verifyGetTables { + () => metadata.getTables("spark_catalog", "db1", "%", Array("VIEW")) + } { getTableResults => + // results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM and TABLE_NAME + assert { + getTableResults.map { result => + (result.TABLE_TYPE, result.TABLE_CAT, result.TABLE_SCHEM, result.TABLE_NAME) + } === Seq(("VIEW", "spark_catalog", "db1", "t1_v")) + } + getTableResults.foreach(verifyEmptyStringFields) + } + + // list tables in the current catalog and schema + conn.setCatalog("spark_catalog") + conn.setSchema("db1") + assert(conn.getCatalog === "spark_catalog") + assert(conn.getSchema === "db1") + + verifyGetTables { + () => metadata.getTables("", "", "%", null) + } { getTableResults => + assert { + getTableResults.map { result => + (result.TABLE_TYPE, result.TABLE_CAT, result.TABLE_SCHEM, result.TABLE_NAME) + } === Seq( + ("TABLE", "spark_catalog", "db1", "t1"), + ("TABLE", "spark_catalog", "db1", "t_2"), + ("VIEW", "spark_catalog", "db1", "t1_v")) + } + getTableResults.foreach(verifyEmptyStringFields) + } + + // list tables with schema pattern and table mame pattern + verifyGetTables { + () => metadata.getTables(null, "db%", "t_", null) + } { getTableResults => + assert { + getTableResults.map { result => + (result.TABLE_TYPE, result.TABLE_CAT, result.TABLE_SCHEM, result.TABLE_NAME) + } === Seq( + ("TABLE", "spark_catalog", "db1", "t1"), + ("TABLE", "spark_catalog", "db_", "t_")) + } + getTableResults.foreach(verifyEmptyStringFields) + } + + // escape _ in schema pattern and table mame pattern + verifyGetTables { + () => metadata.getTables(null, "db\\_", "t\\_", null) + } { getTableResults => + assert { + getTableResults.map { result => + (result.TABLE_TYPE, result.TABLE_CAT, result.TABLE_SCHEM, result.TABLE_NAME) + } === Seq(("TABLE", "spark_catalog", "db_", "t_")) + } + getTableResults.foreach(verifyEmptyStringFields) + } + + // skip testing escape ', % in schema pattern, because Spark SQL does not + // allow using those chars in schema table name. + } + } + } } From d383f260db40e4c7a51cf749137ee72a4f05c369 Mon Sep 17 00:00:00 2001 From: vinodkc Date: Tue, 11 Nov 2025 22:07:02 -0800 Subject: [PATCH 112/400] [SPARK-54207][CONNECT] Supports Date type data in SparkConnectResultSet ### What changes were proposed in this pull request? Add DATE type support to the Spark Connect JDBC client ### Why are the changes needed? DATE is a fundamental SQL type required for JDBC compliance and interoperability. ### Does this PR introduce _any_ user-facing change? Yes, it's part of a new feature under Spark connect JDBC support. ### How was this patch tested? Added new UTs in `SparkConnectJdbcDataTypeSuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #53010 from vinodkc/br_SPARK-54207. Authored-by: vinodkc Signed-off-by: Dongjoon Hyun (cherry picked from commit a8ca6f4ba09994d280b1ae224f180557edd22e26) Signed-off-by: Dongjoon Hyun --- .../client/jdbc/SparkConnectResultSet.scala | 25 +++- .../client/jdbc/util/JdbcTypeUtils.scala | 9 +- .../jdbc/SparkConnectJdbcDataTypeSuite.scala | 109 ++++++++++++++++++ 3 files changed, 135 insertions(+), 8 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index ff02cd73dcc61..5544dee6df5f6 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -140,8 +140,9 @@ class SparkConnectResultSet( override def getBytes(columnIndex: Int): Array[Byte] = throw new SQLFeatureNotSupportedException - override def getDate(columnIndex: Int): Date = - throw new SQLFeatureNotSupportedException + override def getDate(columnIndex: Int): Date = { + getColumnValue(columnIndex, null: Date) { idx => currentRow.getDate(idx) } + } override def getTime(columnIndex: Int): Time = throw new SQLFeatureNotSupportedException @@ -189,7 +190,7 @@ class SparkConnectResultSet( throw new SQLFeatureNotSupportedException override def getDate(columnLabel: String): Date = - throw new SQLFeatureNotSupportedException + getDate(findColumn(columnLabel)) override def getTime(columnLabel: String): Time = throw new SQLFeatureNotSupportedException @@ -496,11 +497,23 @@ class SparkConnectResultSet( override def getArray(columnLabel: String): JdbcArray = throw new SQLFeatureNotSupportedException - override def getDate(columnIndex: Int, cal: Calendar): Date = - throw new SQLFeatureNotSupportedException + override def getDate(columnIndex: Int, cal: Calendar): Date = { + val date = getDate(columnIndex) + if (date == null || cal == null) { + return date + } + + val targetCalendar = cal.clone().asInstanceOf[Calendar] + targetCalendar.setTime(date) + targetCalendar.set(Calendar.HOUR_OF_DAY, 0) + targetCalendar.set(Calendar.MINUTE, 0) + targetCalendar.set(Calendar.SECOND, 0) + targetCalendar.set(Calendar.MILLISECOND, 0) + new Date(targetCalendar.getTimeInMillis) + } override def getDate(columnLabel: String, cal: Calendar): Date = - throw new SQLFeatureNotSupportedException + getDate(findColumn(columnLabel), cal) override def getTime(columnIndex: Int, cal: Calendar): Time = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala index c2b27128caa72..4af67f1582d32 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala @@ -36,6 +36,7 @@ private[jdbc] object JdbcTypeUtils { case DoubleType => Types.DOUBLE case StringType => Types.VARCHAR case _: DecimalType => Types.DECIMAL + case DateType => Types.DATE case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -51,6 +52,7 @@ private[jdbc] object JdbcTypeUtils { case DoubleType => classOf[JDouble].getName case StringType => classOf[String].getName case _: DecimalType => classOf[JBigDecimal].getName + case DateType => classOf[Date].getName case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -58,7 +60,7 @@ private[jdbc] object JdbcTypeUtils { def isSigned(field: StructField): Boolean = field.dataType match { case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | _: DecimalType => true - case NullType | BooleanType | StringType => false + case NullType | BooleanType | StringType | DateType => false case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -74,6 +76,7 @@ private[jdbc] object JdbcTypeUtils { case DoubleType => 15 case StringType => 255 case DecimalType.Fixed(p, _) => p + case DateType => 10 case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -81,7 +84,8 @@ private[jdbc] object JdbcTypeUtils { def getScale(field: StructField): Int = field.dataType match { case FloatType => 7 case DoubleType => 15 - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | StringType => 0 + case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | StringType | + DateType => 0 case DecimalType.Fixed(_, s) => s case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") @@ -96,6 +100,7 @@ private[jdbc] object JdbcTypeUtils { case DoubleType => 24 case StringType => getPrecision(field) + case DateType => 10 // length of `YYYY-MM-DD` // precision + negative sign + leading zero + decimal point, like DECIMAL(5,5) = -0.12345 case DecimalType.Fixed(p, s) if p == s => p + 3 // precision + negative sign, like DECIMAL(5,0) = -12345 diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index 217142287b130..1820f30732daf 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -301,4 +301,113 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess assert(exception.getMessage() === "JDBC Statement is closed.") } } + + test("get date type") { + withExecuteQuery("SELECT date '2023-11-15'") { rs => + assert(rs.next()) + assert(rs.getDate(1) === java.sql.Date.valueOf("2023-11-15")) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "DATE '2023-11-15'") + assert(metaData.getColumnLabel(1) === "DATE '2023-11-15'") + assert(metaData.getColumnType(1) === Types.DATE) + assert(metaData.getColumnTypeName(1) === "DATE") + assert(metaData.getColumnClassName(1) === "java.sql.Date") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 10) + assert(metaData.getScale(1) === 0) + assert(metaData.getColumnDisplaySize(1) === 10) + } + } + + test("get date type with null") { + withExecuteQuery("SELECT cast(null as date)") { rs => + assert(rs.next()) + assert(rs.getDate(1) === null) + assert(rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "CAST(NULL AS DATE)") + assert(metaData.getColumnLabel(1) === "CAST(NULL AS DATE)") + assert(metaData.getColumnType(1) === Types.DATE) + assert(metaData.getColumnTypeName(1) === "DATE") + assert(metaData.getColumnClassName(1) === "java.sql.Date") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 10) + assert(metaData.getScale(1) === 0) + assert(metaData.getColumnDisplaySize(1) === 10) + } + } + + test("get date type by column label") { + withExecuteQuery("SELECT date '2025-11-15' as test_date") { rs => + assert(rs.next()) + assert(rs.getDate("test_date") === java.sql.Date.valueOf("2025-11-15")) + assert(!rs.wasNull) + assert(!rs.next()) + } + } + + test("get date type with calendar by column index") { + withExecuteQuery("SELECT date '2025-11-15'") { rs => + assert(rs.next()) + + val calUTC = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) + val dateUTC = rs.getDate(1, calUTC) + assert(dateUTC !== null) + assert(!rs.wasNull) + + val calPST = java.util.Calendar.getInstance( + java.util.TimeZone.getTimeZone("America/Los_Angeles")) + val datePST = rs.getDate(1, calPST) + assert(datePST !== null) + assert(!rs.wasNull) + assert(!rs.next()) + } + } + + test("get date type with calendar by column label") { + withExecuteQuery("SELECT date '2025-11-15' as test_date") { rs => + assert(rs.next()) + + val cal = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) + val date = rs.getDate("test_date", cal) + assert(date !== null) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "test_date") + assert(metaData.getColumnLabel(1) === "test_date") + } + } + + test("get date type with calendar for null value") { + withExecuteQuery("SELECT cast(null as date)") { rs => + assert(rs.next()) + + val cal = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) + val date = rs.getDate(1, cal) + assert(date === null) + assert(rs.wasNull) + assert(!rs.next()) + } + } + + test("get date type with null calendar") { + withExecuteQuery("SELECT date '2025-11-15'") { rs => + assert(rs.next()) + + val date = rs.getDate(1, null) + assert(date === java.sql.Date.valueOf("2025-11-15")) + assert(!rs.wasNull) + assert(!rs.next()) + } + } } From 37d6b960d1efe51e93698d1639fb7085d5e6cccb Mon Sep 17 00:00:00 2001 From: Ganesha S Date: Wed, 12 Nov 2025 05:50:40 -0800 Subject: [PATCH 113/400] [SPARK-54130][SQL] Add detailed error messages for catalog assertion failures ### What changes were proposed in this pull request? This PR enhances error reporting in the catalog layer by adding detailed, user-friendly assertion messages for various metadata validation failures. The changes add informative error messages to 11 bare `assert()` statements across the catalog codebase. **Files modified:** - `SessionCatalog.scala`: Enhanced 5 assertions with detailed error messages - `interface.scala`: Enhanced 2 assertions (UnresolvedCatalogRelation, HiveTableRelation) - `SQLFunction.scala`: Enhanced 2 assertions for function validation - `InMemoryCatalog.scala`: Enhanced 2 assertions for table operations - `SessionCatalogSuite.scala`: Added 6 new unit tests to verify error messages **Enhanced assertions include:** - Table identifier validation (missing database names) - SQL function body validation (missing expression/query text) - SQL function type validation (mismatched function type and return type) - Function plan method validation (wrong method called for function type) - Table function return parameter validation ### Why are the changes needed? Currently, when metadata validation fails, assertions fail with generic "assertion failed" messages that provide no context about: - Which table/function has the problem - What the actual vs expected values are - What specific validation failed - How to resolve the issue This makes debugging production issues very difficult, especially for: - Missing database names in table identifiers - Malformed SQL function definitions - Type mismatches in SQL functions - Internal method call errors The enhanced error messages provide all necessary information to quickly identify and resolve issues. ### Does this PR introduce _any_ user-facing change? Yes. Users will now see detailed error messages instead of generic assertion failures. **Before:** assertion failed **After:** assertion failed: Table identifier test_table is missing database name. UnresolvedCatalogRelation requires a fully qualified table identifier with database. **Example for SQL functions:** assertion failed: SQL function 'test_func' is missing function body. Either exprText or queryText must be defined. Found: exprText=None, queryText=None. ### How was this patch tested? Added 6 new unit tests in `SessionCatalogSuite.scala` that: - Create scenarios that trigger each assertion - Verify that AssertionError is thrown - Validate the complete error message matches expected format **Test coverage:** - `test("UnresolvedCatalogRelation requires database in identifier")` - `test("HiveTableRelation requires database in identifier")` - `test("SQLFunction requires either exprText or queryText")` - `test("SQLFunction return type must match function type")` - `test("InMemoryCatalog.createTable requires database in identifier")` - `test("InMemoryCatalog.alterTable requires database in identifier")` All existing tests continue to pass. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude 3.5 Sonnet (Anthropic) Closes #52826 from ganeshashree/SPARK-54130. Authored-by: Ganesha S Signed-off-by: Wenchen Fan (cherry picked from commit 0c715b4de87fb7a6da2b9e2cf1a1ca68f37885a1) Signed-off-by: Wenchen Fan --- .../catalyst/catalog/InMemoryCatalog.scala | 10 +- .../sql/catalyst/catalog/SQLFunction.scala | 11 +- .../sql/catalyst/catalog/SessionCatalog.scala | 29 ++- .../sql/catalyst/catalog/interface.scala | 8 +- .../catalog/SessionCatalogSuite.scala | 178 ++++++++++++++++++ 5 files changed, 224 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 5d0184579faac..f5c732ee1412c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -192,7 +192,10 @@ class InMemoryCatalog( override def createTable( tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = synchronized { - assert(tableDefinition.identifier.database.isDefined) + assert(tableDefinition.identifier.database.isDefined, + "Table identifier " + tableDefinition.identifier.quotedString + + " is missing database name. " + + "Cannot create table without a database specified.") val db = tableDefinition.identifier.database.get requireDbExists(db) val table = tableDefinition.identifier.table @@ -313,7 +316,10 @@ class InMemoryCatalog( } override def alterTable(tableDefinition: CatalogTable): Unit = synchronized { - assert(tableDefinition.identifier.database.isDefined) + assert(tableDefinition.identifier.database.isDefined, + "Table identifier " + tableDefinition.identifier.quotedString + + " is missing database name. " + + "Cannot alter table without a database specified.") val db = tableDefinition.identifier.database.get requireTableExists(db, tableDefinition.identifier.table) val updatedProperties = tableDefinition.properties.filter(kv => kv._1 != "comment") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala index f2fd3b90f6468..84d87fab8b060 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala @@ -61,8 +61,15 @@ case class SQLFunction( owner: Option[String] = None, createTimeMs: Long = System.currentTimeMillis) extends UserDefinedFunction { - assert(exprText.nonEmpty || queryText.nonEmpty) - assert((isTableFunc && returnType.isRight) || (!isTableFunc && returnType.isLeft)) + assert(exprText.nonEmpty || queryText.nonEmpty, + "SQL function '" + name + "' is missing function body. " + + "Either exprText or queryText must be defined. " + + "Found: exprText=" + exprText + ", queryText=" + queryText + ".") + assert((isTableFunc && returnType.isRight) || (!isTableFunc && returnType.isLeft), + "SQL function '" + name + "' has mismatched function type and return type. " + + "isTableFunc=" + isTableFunc + ", returnType.isRight=" + returnType.isRight + ", " + + "returnType.isLeft=" + returnType.isLeft + ". " + + "Table functions require Right[StructType] and scalar functions require Left[DataType].") import SQLFunction._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index c351aacd45a4e..be90c7ad3656c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1663,7 +1663,9 @@ class SessionCatalog( .putString("__funcInputAlias", "true") .build() } - assert(!function.isTableFunc) + assert(!function.isTableFunc, + "Function '" + function.name + "' is a table function. " + + "Use makeSQLTableFunctionPlan() instead of makeSQLFunctionPlan().") val funcName = function.name.funcName // Use captured SQL configs when parsing a SQL function. @@ -1674,7 +1676,10 @@ class SessionCatalog( val inputParam = function.inputParam val returnType = function.getScalarFuncReturnType val (expression, query) = function.getExpressionAndQuery(parser, isTableFunc = false) - assert(expression.isDefined || query.isDefined) + assert(expression.isDefined || query.isDefined, + "SQL function '" + function.name + "' could not be parsed. " + + "Neither expression nor query could be extracted from function body. " + + "exprText=" + function.exprText + ", queryText=" + function.queryText + ".") // Check function arguments val paramSize = inputParam.map(_.size).getOrElse(0) @@ -1763,12 +1768,17 @@ class SessionCatalog( function: SQLFunction, input: Seq[Expression], outputAttrs: Seq[Attribute]): LogicalPlan = { - assert(function.isTableFunc) + assert(function.isTableFunc, + "Function '" + function.name + "' is a scalar function. " + + "Use makeSQLFunctionPlan() instead of makeSQLTableFunctionPlan().") val funcName = function.name.funcName val inputParam = function.inputParam val returnParam = function.getTableFuncReturnCols val (_, query) = function.getExpressionAndQuery(parser, isTableFunc = true) - assert(query.isDefined) + assert(query.isDefined, + "SQL table function '" + function.name + "' could not be parsed. " + + "Query could not be extracted from function body. " + + "queryText=" + function.queryText + ".") // Check function arguments val paramSize = inputParam.map(_.size).getOrElse(0) @@ -1807,7 +1817,12 @@ class SessionCatalog( query.get } - assert(returnParam.length == outputAttrs.length) + assert(returnParam.length == outputAttrs.length, + "SQL table function '" + function.name + "' has mismatched return columns. " + + "Expected " + outputAttrs.length + " output attributes but found " + + returnParam.length + " return parameters. " + + "Return parameters: [" + returnParam.fieldNames.mkString(", ") + "], " + + "Output attributes: [" + outputAttrs.map(_.name).mkString(", ") + "].") val output = returnParam.fields.zipWithIndex.map { case (param, i) => // Since we cannot get the output of a unresolved logical plan, we need // to reference the output column of the lateral join by its position. @@ -2390,7 +2405,9 @@ class SessionCatalog( requireTableNotExists(newName) val oldTable = getTableMetadata(oldName) if (oldTable.tableType == CatalogTableType.MANAGED) { - assert(oldName.database.nonEmpty) + assert(oldName.database.nonEmpty, + "Table identifier " + oldName.quotedString + " is missing database name. " + + "Managed tables must have a database defined.") val databaseLocation = externalCatalog.getDatabase(oldName.database.get).locationUri val newTableLocation = new Path(new Path(databaseLocation), format(newName.table)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 07d26813be940..eab99a96f4c3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -1070,7 +1070,9 @@ case class UnresolvedCatalogRelation( tableMeta: CatalogTable, options: CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty(), override val isStreaming: Boolean = false) extends UnresolvedLeafNode { - assert(tableMeta.identifier.database.isDefined) + assert(tableMeta.identifier.database.isDefined, + "Table identifier " + tableMeta.identifier.quotedString + " is missing database name. " + + "UnresolvedCatalogRelation requires a fully qualified table identifier with database.") } /** @@ -1097,7 +1099,9 @@ case class HiveTableRelation( tableStats: Option[Statistics] = None, @transient prunedPartitions: Option[Seq[CatalogTablePartition]] = None) extends LeafNode with MultiInstanceRelation with NormalizeableRelation { - assert(tableMeta.identifier.database.isDefined) + assert(tableMeta.identifier.database.isDefined, + "Table identifier " + tableMeta.identifier.quotedString + " is missing database name. " + + "HiveTableRelation requires a fully qualified table identifier with database.") assert(DataTypeUtils.sameType(tableMeta.partitionSchema, partitionCols.toStructType)) assert(DataTypeUtils.sameType(tableMeta.dataSchema, dataCols.toStructType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 309d518f5ef4f..92a7154c59cb4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils class InMemorySessionCatalogSuite extends SessionCatalogSuite { protected val utils = new CatalogTestUtils { @@ -2091,4 +2092,181 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(exception.getMessage.matches(expectedPattern)) } } + + test("UnresolvedCatalogRelation requires database in identifier") { + val catalog = new SessionCatalog(newEmptyCatalog()) + catalog.createDatabase(newDb("default"), ignoreIfExists = true) + val db = "test_db" + catalog.createDatabase(newDb(db), ignoreIfExists = true) + + // Create a table with database + val validTable = CatalogTable( + identifier = TableIdentifier("test_table", Some(db)), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("id", IntegerType) + ) + catalog.createTable(validTable, ignoreIfExists = false) + + // Try to create UnresolvedCatalogRelation without database - should fail + val tableMetaWithoutDb = validTable.copy( + identifier = TableIdentifier("test_table", None) + ) + + val exception = intercept[AssertionError] { + UnresolvedCatalogRelation(tableMetaWithoutDb) + } + + val expectedMessage = + "assertion failed: Table identifier `test_table` is missing database name. " + + "UnresolvedCatalogRelation requires a fully qualified table identifier with database." + assert(exception.getMessage === expectedMessage) + } + + test("HiveTableRelation requires database in identifier") { + val catalog = new SessionCatalog(newEmptyCatalog()) + catalog.createDatabase(newDb("default"), ignoreIfExists = true) + val db = "test_db" + catalog.createDatabase(newDb(db), ignoreIfExists = true) + + // Create a table with database + val validTable = CatalogTable( + identifier = TableIdentifier("test_table", Some(db)), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("id", IntegerType) + .add("name", StringType) + ) + + // Try to create HiveTableRelation without database - should fail + val tableMetaWithoutDb = validTable.copy( + identifier = TableIdentifier("test_table", None) + ) + + val exception = intercept[AssertionError] { + HiveTableRelation( + tableMetaWithoutDb, + Seq(AttributeReference("id", IntegerType)()), + Seq.empty + ) + } + + val expectedMessage = + "assertion failed: Table identifier `test_table` is missing database name. " + + "HiveTableRelation requires a fully qualified table identifier with database." + assert(exception.getMessage === expectedMessage) + } + + test("SQLFunction requires either exprText or queryText") { + // Test case 1: Neither exprText nor queryText provided + val exception1 = intercept[AssertionError] { + SQLFunction( + name = FunctionIdentifier("test_func"), + inputParam = None, + returnType = scala.util.Left(IntegerType), + exprText = None, + queryText = None, + comment = None, + deterministic = Some(true), + containsSQL = Some(false), + isTableFunc = false, + properties = Map.empty + ) + } + + val expectedMessage = "assertion failed: SQL function 'test_func' is missing function body. " + + "Either exprText or queryText must be defined. " + + "Found: exprText=None, queryText=None." + assert(exception1.getMessage === expectedMessage) + } + + test("SQLFunction return type must match function type") { + // Test case: isTableFunc=true but returnType is Left (scalar type) + val exception = intercept[AssertionError] { + SQLFunction( + name = FunctionIdentifier("test_func"), + inputParam = None, + returnType = scala.util.Left(IntegerType), // Scalar return type + exprText = Some("SELECT 1"), + queryText = None, + comment = None, + deterministic = Some(true), + containsSQL = Some(true), + isTableFunc = true, // But marked as table function + properties = Map.empty + ) + } + + val expectedMessage = + "assertion failed: SQL function 'test_func' has mismatched function type " + + "and return type. " + + "isTableFunc=true, returnType.isRight=false, returnType.isLeft=true. " + + "Table functions require Right[StructType] and scalar functions require Left[DataType]." + assert(exception.getMessage === expectedMessage) + } + + test("InMemoryCatalog.createTable requires database in identifier") { + val catalog = new InMemoryCatalog() + val db = "test_db" + val dbDefinition = CatalogDatabase( + name = db, + description = "test database", + locationUri = Utils.createTempDir().toURI, + properties = Map.empty + ) + catalog.createDatabase(dbDefinition, ignoreIfExists = false) + + // Try to create table without database - should fail + val tableWithoutDb = CatalogTable( + identifier = TableIdentifier("test_table", None), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("id", IntegerType) + ) + + val exception = intercept[AssertionError] { + catalog.createTable(tableWithoutDb, ignoreIfExists = false) + } + + val expectedMessage = + "assertion failed: Table identifier `test_table` is missing database name. " + + "Cannot create table without a database specified." + assert(exception.getMessage === expectedMessage) + } + + test("InMemoryCatalog.alterTable requires database in identifier") { + val catalog = new InMemoryCatalog() + val db = "test_db" + val dbDefinition = CatalogDatabase( + name = db, + description = "test database", + locationUri = Utils.createTempDir().toURI, + properties = Map.empty + ) + catalog.createDatabase(dbDefinition, ignoreIfExists = false) + + // First create a valid table + val validTable = CatalogTable( + identifier = TableIdentifier("test_table", Some(db)), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("id", IntegerType) + ) + catalog.createTable(validTable, ignoreIfExists = false) + + // Try to alter table with identifier without database - should fail + val tableWithoutDb = validTable.copy( + identifier = TableIdentifier("test_table", None) + ) + + val exception = intercept[AssertionError] { + catalog.alterTable(tableWithoutDb) + } + + val expectedMessage = + "assertion failed: Table identifier `test_table` is missing database name. " + + "Cannot alter table without a database specified." + assert(exception.getMessage === expectedMessage) + } } From 2ee1dcb150c2d73aa05c101c1993b93e8502fb8f Mon Sep 17 00:00:00 2001 From: Alex Khakhlyuk Date: Wed, 12 Nov 2025 10:18:09 -0400 Subject: [PATCH 114/400] [SPARK-53917][CONNECT] Support large local relations - follow-ups ### What changes were proposed in this pull request? This PR adds several fixes and improvements over https://github.com/apache/spark/pull/52613 which added support for 2GB+ local relations in Spark Connect. #### Uploading batches of chunks Currently, before caching the local relation on the server via `ChunkedCachedLocalRelation`, the client materializes all chunks in memory (1 schema chunk and N data chunks). This can lead to high memory pressure on the client when uploading very large local relations. In this PR, I'm changing how the client uploads the local relation. Instead of materializing all chunks in memory, the client will materialize a batch of chunks in memory, upload the batch of chunks to the server, and proceed to collecting the next batch of chunks. The size of the batch of chunks is controlled via `spark.sql.session.localRelationBatchOfChunksSizeBytes` (1GB by default). This way, the uploading mechanism only consumes 1GB of memory at each point in time. Alternatives to this approach are: a) uploading each chunk separately - would require one pair of `ArtifactStatuses` and `AddArtifactsRequest` RPC calls for each chunk, which is inefficient. b) (current implementation) materializing all chunks in memory and uploading them via a single pair of `ArtifactStatuses` and `AddArtifactsRequest` RPC calls. This can lead to high memory pressure on the client. Uploading batches of chunks is a middle-ground solution. Changes are implemented both for the python and scala clients. #### Minor fixes and improvements - Replace `ArraySeq.unsafeWrapArray(data.map(_.copy()).toArray))` with `data.map(_.copy()).toArray.toImmutableArraySeq` in `SparkConnectPlanner.scala`. The latter is compatible with both scala 2.13 and scala 2.12 and is consistent with how arrays are converted to sequences in other places in the code base. - Improved asserts and tests in the python client. ### Why are the changes needed? Reduce memory pressure in the spark connect python and scala clients when uploading very large local relations to the server. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52973 from khakhlyuk/largelocalrelations-followup. Authored-by: Alex Khakhlyuk Signed-off-by: Herman van Hovell (cherry picked from commit 40ba971b7319d74670ba86cc1f280a8a0f7a1dbb) Signed-off-by: Herman van Hovell --- python/pyspark/sql/connect/plan.py | 52 ++++++------ python/pyspark/sql/connect/session.py | 66 ++++++++++++--- python/pyspark/sql/tests/arrow/test_arrow.py | 8 +- .../spark/sql/internal/SqlApiConf.scala | 2 + .../spark/sql/internal/SqlApiConfHelper.scala | 2 + .../apache/spark/sql/internal/SQLConf.scala | 19 ++++- .../spark/sql/connect/SparkSession.scala | 80 +++++++++++++------ .../connect/client/SparkConnectClient.scala | 20 ----- .../connect/planner/SparkConnectPlanner.scala | 3 +- 9 files changed, 166 insertions(+), 86 deletions(-) diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 02fe7176b6fee..6630d96f21ded 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -24,6 +24,7 @@ from typing import ( Any, + Iterator, List, Optional, Type, @@ -437,7 +438,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: return plan def _serialize_table(self) -> bytes: - assert self._table is not None + assert self._table is not None, "table cannot be None" sink = pa.BufferOutputStream() with pa.ipc.new_stream(sink, self._table.schema) as writer: batches = self._table.to_batches() @@ -449,7 +450,7 @@ def _serialize_table_chunks( self, max_chunk_size_rows: int, max_chunk_size_bytes: int, - ) -> list[bytes]: + ) -> Iterator[bytes]: """ Serialize the table into multiple chunks, each up to max_chunk_size_bytes bytes and max_chunk_size_rows rows. @@ -457,49 +458,52 @@ def _serialize_table_chunks( This method processes the table in fixed-size batches (1024 rows) for efficiency, matching the Scala implementation's batchSizeCheckInterval. + + Yields chunks one at a time to avoid materializing all chunks in memory. """ - assert self._table is not None - chunks = [] + assert self._table is not None, "table cannot be None" + assert self._table.num_rows > 0, "table must have at least one row" schema = self._table.schema - # Calculate schema serialization size once - schema_buffer = pa.BufferOutputStream() - with pa.ipc.new_stream(schema_buffer, schema): - pass # Just write schema - schema_size = len(schema_buffer.getvalue()) + # Calculate schema serialization size once (empty table = just schema) + schema_size = len(self._serialize_batches_to_ipc([], schema)) current_batches: list[pa.RecordBatch] = [] current_size = schema_size for batch in self._table.to_batches(max_chunksize=min(1024, max_chunk_size_rows)): + # Approximate batch size using raw column data (fast, ignores IPC overhead). + # Calculating the real batch size of the IPC stream would require serializing each + # batch separately, which adds overhead. batch_size = sum(arr.nbytes for arr in batch.columns) # If this batch would exceed limit and we have data, flush current chunk - if current_size > schema_size and current_size + batch_size > max_chunk_size_bytes: - combined = pa.Table.from_batches(current_batches, schema=schema) - sink = pa.BufferOutputStream() - with pa.ipc.new_stream(sink, schema) as writer: - writer.write_table(combined) - chunks.append(sink.getvalue().to_pybytes()) + if len(current_batches) > 0 and current_size + batch_size > max_chunk_size_bytes: + yield self._serialize_batches_to_ipc(current_batches, schema) current_batches = [] current_size = schema_size current_batches.append(batch) current_size += batch_size - # Flush remaining batches - if current_batches: - combined = pa.Table.from_batches(current_batches, schema=schema) - sink = pa.BufferOutputStream() - with pa.ipc.new_stream(sink, schema) as writer: - writer.write_table(combined) - chunks.append(sink.getvalue().to_pybytes()) + # Flush remaining batches (guaranteed to have at least one due to assertion) + yield self._serialize_batches_to_ipc(current_batches, schema) - return chunks + def _serialize_batches_to_ipc( + self, + batches: list[pa.RecordBatch], + schema: pa.Schema, + ) -> bytes: + """Helper method to serialize Arrow batches to IPC stream format.""" + combined = pa.Table.from_batches(batches, schema=schema) + sink = pa.BufferOutputStream() + with pa.ipc.new_stream(sink, schema) as writer: + writer.write_table(combined) + return sink.getvalue().to_pybytes() def _serialize_schema(self) -> bytes: # the server uses UTF-8 for decoding the schema - assert self._schema is not None + assert self._schema is not None, "schema cannot be None" return self._schema.encode("utf-8") def serialize(self, session: "SparkConnectClient") -> bytes: diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 21a7c8329a354..ac1d1f5681e3e 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -29,6 +29,7 @@ from typing import ( Optional, Any, + Iterator, Union, Dict, List, @@ -537,6 +538,7 @@ def createDataFrame( "spark.sql.session.localRelationCacheThreshold", "spark.sql.session.localRelationChunkSizeRows", "spark.sql.session.localRelationChunkSizeBytes", + "spark.sql.session.localRelationBatchOfChunksSizeBytes", "spark.sql.execution.pandas.convertToArrowArraySafely", "spark.sql.execution.pandas.inferPandasDictAsMap", "spark.sql.pyspark.inferNestedDictAsStruct.enabled", @@ -772,10 +774,16 @@ def createDataFrame( max_chunk_size_bytes = int( configs["spark.sql.session.localRelationChunkSizeBytes"] # type: ignore[arg-type] ) + max_batch_of_chunks_size_bytes = int( + configs["spark.sql.session.localRelationBatchOfChunksSizeBytes"] # type: ignore[arg-type] # noqa: E501 + ) plan: LogicalPlan = local_relation if cache_threshold <= _table.nbytes: plan = self._cache_local_relation( - local_relation, max_chunk_size_rows, max_chunk_size_bytes + local_relation, + max_chunk_size_rows, + max_chunk_size_bytes, + max_batch_of_chunks_size_bytes, ) df = DataFrame(plan, self) @@ -1054,30 +1062,62 @@ def _cache_local_relation( local_relation: LocalRelation, max_chunk_size_rows: int, max_chunk_size_bytes: int, + max_batch_of_chunks_size_bytes: int, ) -> ChunkedCachedLocalRelation: """ Cache the local relation at the server side if it has not been cached yet. - Should only be called on LocalRelations with _table set. + This method serializes the input local relation into multiple data chunks and + a schema chunk (if the schema is available) and uploads these chunks as artifacts + to the server. + + The method collects a batch of chunks of size up to max_batch_of_chunks_size_bytes and + uploads them together to the server. + Uploading each chunk separately would require an additional RPC call for each chunk. + Uploading all chunks together would require materializing all chunks in memory which + may cause high memory usage on the client. + Uploading batches of chunks is the middle-ground solution. + + Should only be called on a LocalRelation with a non-empty _table. """ - assert local_relation._table is not None + assert local_relation._table is not None, "table cannot be None" has_schema = local_relation._schema is not None - # Serialize table into chunks - data_chunks = local_relation._serialize_table_chunks( - max_chunk_size_rows, max_chunk_size_bytes + hashes = [] + current_batch = [] + current_batch_size = 0 + if has_schema: + schema_chunk = local_relation._serialize_schema() + current_batch.append(schema_chunk) + current_batch_size += len(schema_chunk) + + data_chunks: Iterator[bytes] = local_relation._serialize_table_chunks( + max_chunk_size_rows, min(max_chunk_size_bytes, max_batch_of_chunks_size_bytes) ) - blobs = data_chunks.copy() # Start with data chunks - if has_schema: - blobs.append(local_relation._serialize_schema()) + for chunk in data_chunks: + chunk_size = len(chunk) - hashes = self._client.cache_artifacts(blobs) + # Check if adding this chunk would exceed batch size + if ( + len(current_batch) > 0 + and current_batch_size + chunk_size > max_batch_of_chunks_size_bytes + ): + hashes += self._client.cache_artifacts(current_batch) + # start a new batch + current_batch = [] + current_batch_size = 0 - # Extract data hashes and schema hash - data_hashes = hashes[: len(data_chunks)] - schema_hash = hashes[len(data_chunks)] if has_schema else None + current_batch.append(chunk) + current_batch_size += chunk_size + hashes += self._client.cache_artifacts(current_batch) + if has_schema: + schema_hash = hashes[0] + data_hashes = hashes[1:] + else: + schema_hash = None + data_hashes = hashes return ChunkedCachedLocalRelation(data_hashes, schema_hash) def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None: diff --git a/python/pyspark/sql/tests/arrow/test_arrow.py b/python/pyspark/sql/tests/arrow/test_arrow.py index 14f8fbe33c8e1..79d8bf77d9d5b 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow.py +++ b/python/pyspark/sql/tests/arrow/test_arrow.py @@ -438,10 +438,12 @@ def check_cached_local_relation_changing_values(self): assert not df.filter(df["col2"].endswith(suffix)).isEmpty() def check_large_cached_local_relation_same_values(self): - data = [("C000000032", "R20", 0.2555)] * 500_000 + row_count = 500_000 + data = [("C000000032", "R20", 0.2555)] * row_count pdf = pd.DataFrame(data=data, columns=["Contrat", "Recommandation", "Distance"]) - df = self.spark.createDataFrame(pdf) - df.collect() + for _ in range(2): + df = self.spark.createDataFrame(pdf) + assert df.count() == row_count def test_toArrow_keep_utc_timezone(self): df = self.spark.createDataFrame(self.data, schema=self.schema) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index f715f8f9ed8cd..0973750c65ce4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -67,6 +67,8 @@ private[sql] object SqlApiConf { SqlApiConfHelper.LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY val LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY: String = SqlApiConfHelper.LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY + val LOCAL_RELATION_BATCH_OF_CHUNKS_SIZE_BYTES_KEY: String = + SqlApiConfHelper.LOCAL_RELATION_BATCH_OF_CHUNKS_SIZE_BYTES_KEY val PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY: String = SqlApiConfHelper.PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY val PARSER_DFA_CACHE_FLUSH_RATIO_KEY: String = diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala index b839caba3f547..4fcc2f4e150d1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala @@ -35,6 +35,8 @@ private[sql] object SqlApiConfHelper { val LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY: String = "spark.sql.session.localRelationChunkSizeRows" val LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY: String = "spark.sql.session.localRelationChunkSizeBytes" + val LOCAL_RELATION_BATCH_OF_CHUNKS_SIZE_BYTES_KEY: String = + "spark.sql.session.localRelationBatchOfChunksSizeBytes" val ARROW_EXECUTION_USE_LARGE_VAR_TYPES = "spark.sql.execution.arrow.useLargeVarTypes" val PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY: String = "spark.sql.parser.parserDfaCacheFlushThreshold" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6ab081d54dc47..b3a251fc671e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6055,7 +6055,9 @@ object SQLConf { .doc("The chunk size in bytes when splitting ChunkedCachedLocalRelation.data " + "into batches. A new chunk is created when either " + "spark.sql.session.localRelationChunkSizeBytes " + - "or spark.sql.session.localRelationChunkSizeRows is reached.") + "or spark.sql.session.localRelationChunkSizeRows is reached. " + + "Limited by the spark.sql.session.localRelationBatchOfChunksSizeBytes, " + + "a minimum of the two confs is used to determine the chunk size.") .version("4.1.0") .longConf .checkValue(_ > 0, "The chunk size in bytes must be positive") @@ -6079,6 +6081,21 @@ object SQLConf { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("3GB") + val LOCAL_RELATION_BATCH_OF_CHUNKS_SIZE_BYTES = + buildConf(SqlApiConfHelper.LOCAL_RELATION_BATCH_OF_CHUNKS_SIZE_BYTES_KEY) + .internal() + .doc("Limit on how much memory the client can use when uploading a local relation to the " + + "server. The client collects multiple local relation chunks into a single batch in " + + "memory until the limit is reached, then uploads the batch to the server. " + + "This helps reduce memory pressure on the client when dealing with very large local " + + "relations because the client does not have to materialize all chunks in memory. " + + "Limits the spark.sql.session.localRelationChunkSizeBytes, " + + "a minimum of the two confs is used to determine the chunk size.") + .version("4.1.0") + .longConf + .checkValue(_ > 0, "The batch size in bytes must be positive") + .createWithDefault(1 * 1024 * 1024 * 1024L) + val DECORRELATE_JOIN_PREDICATE_ENABLED = buildConf("spark.sql.optimizer.decorrelateJoinPredicate.enabled") .internal() diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala index 0d9d4e5d60f0a..daa2cc2001e42 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala @@ -118,39 +118,73 @@ class SparkSession private[sql] ( newDataset(encoder) { builder => if (data.nonEmpty) { val threshold = conf.get(SqlApiConf.LOCAL_RELATION_CACHE_THRESHOLD_KEY).toInt - val maxRecordsPerBatch = conf.get(SqlApiConf.LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY).toInt - val maxBatchSize = conf.get(SqlApiConf.LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY).toInt + val maxChunkSizeRows = conf.get(SqlApiConf.LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY).toInt + val maxChunkSizeBytes = conf.get(SqlApiConf.LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY).toInt + val maxBatchOfChunksSize = + conf.get(SqlApiConf.LOCAL_RELATION_BATCH_OF_CHUNKS_SIZE_BYTES_KEY).toLong + // Serialize with chunking support val it = ArrowSerializer.serialize( data, encoder, allocator, - maxRecordsPerBatch = maxRecordsPerBatch, - maxBatchSize = maxBatchSize, + maxRecordsPerBatch = maxChunkSizeRows, + maxBatchSize = math.min(maxChunkSizeBytes, maxBatchOfChunksSize), timeZoneId = timeZoneId, largeVarTypes = largeVarTypes, - batchSizeCheckInterval = math.min(1024, maxRecordsPerBatch)) + batchSizeCheckInterval = math.min(1024, maxChunkSizeRows)) + + try { + val schemaBytes = encoder.schema.json.getBytes + // Schema is the first chunk, data chunks follow from the iterator + val currentBatch = scala.collection.mutable.ArrayBuffer[Array[Byte]](schemaBytes) + var totalChunks = 1 + var currentBatchSize = schemaBytes.length.toLong + var totalSize = currentBatchSize + + // store all hashes of uploaded chunks. The first hash is schema, rest are data hashes + val allHashes = scala.collection.mutable.ArrayBuffer[String]() + while (it.hasNext) { + val chunk = it.next() + val chunkSize = chunk.length + totalChunks += 1 + totalSize += chunkSize + + // Check if adding this chunk would exceed batch size + if (currentBatchSize + chunkSize > maxBatchOfChunksSize) { + // Upload current batch + allHashes ++= client.artifactManager.cacheArtifacts(currentBatch.toArray) + // Start new batch + currentBatch.clear() + currentBatchSize = 0 + } - val chunks = - try { - it.toArray - } finally { - it.close() + currentBatch += chunk + currentBatchSize += chunkSize } - // If we got multiple chunks or a single large chunk, use ChunkedCachedLocalRelation - val totalSize = chunks.map(_.length).sum - if (chunks.length > 1 || totalSize > threshold) { - val (dataHashes, schemaHash) = client.cacheLocalRelation(chunks, encoder.schema.json) - builder.getChunkedCachedLocalRelationBuilder - .setSchemaHash(schemaHash) - .addAllDataHashes(dataHashes.asJava) - } else { - // Small data, use LocalRelation directly - val arrowData = ByteString.copyFrom(chunks(0)) - builder.getLocalRelationBuilder - .setSchema(encoder.schema.json) - .setData(arrowData) + // Decide whether to use LocalRelation or ChunkedCachedLocalRelation + if (totalChunks == 2 && totalSize <= threshold) { + // Schema + single small data chunk: use LocalRelation with inline data + val arrowData = ByteString.copyFrom(currentBatch.last) + builder.getLocalRelationBuilder + .setSchema(encoder.schema.json) + .setData(arrowData) + } else { + // Multiple data chunks or large data: use ChunkedCachedLocalRelation + // Upload remaining batch + allHashes ++= client.artifactManager.cacheArtifacts(currentBatch.toArray) + + // First hash is schema, rest are data + val schemaHash = allHashes.head + val dataHashes = allHashes.tail + + builder.getChunkedCachedLocalRelationBuilder + .setSchemaHash(schemaHash) + .addAllDataHashes(dataHashes.asJava) + } + } finally { + it.close() } } else { builder.getLocalRelationBuilder diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index e5fd16a7c2612..ee42a873787f9 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -428,26 +428,6 @@ private[sql] class SparkConnectClient( channel.shutdownNow() } - /** - * Cache the given local relation Arrow stream from a local file and return its hashes. The file - * is streamed in chunks and does not need to fit in memory. - * - * This method batches artifact status checks and uploads to minimize RPC overhead. - */ - private[sql] def cacheLocalRelation( - data: Array[Array[Byte]], - schema: String): (Seq[String], String) = { - val schemaBytes = schema.getBytes - val allBlobs = data :+ schemaBytes - val allHashes = artifactManager.cacheArtifacts(allBlobs) - - // Last hash is the schema hash, rest are data hashes - val dataHashes = allHashes.dropRight(1) - val schemaHash = allHashes.last - - (dataHashes, schemaHash) - } - /** * Clone this client session, creating a new session with the same configuration and shared * state as the current session but with independent runtime state. diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 8f8e6261066f4..8bc33c41b3a30 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.connect.planner import java.util.{HashMap, Properties, UUID} -import scala.collection.immutable.ArraySeq import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.util.Try @@ -1608,7 +1607,7 @@ class SparkConnectPlanner( schemaOpt match { case None => - logical.LocalRelation(attributes, ArraySeq.unsafeWrapArray(data.map(_.copy()).toArray)) + logical.LocalRelation(attributes, data.map(_.copy()).toArray.toImmutableArraySeq) case Some(schema) => def normalize(dt: DataType): DataType = dt match { case udt: UserDefinedType[_] => normalize(udt.sqlType) From 51d7a20d37b756645290ef9c63f938c60fb1a055 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Wed, 12 Nov 2025 07:43:36 -0800 Subject: [PATCH 115/400] [SPARK-54306] Annotate Variant columns with Variant logical type annotation ### What changes were proposed in this pull request? This PR makes changes to the parquet writer to make it annotate variant columns with the parquet variant logical type annotation. ### Why are the changes needed? The Parquet spec has formally adopted the Variant logical type, and therefore, Variant columns must be properly annotated in Spark 4.1.0 which depends on Parquet-java 1.16.0 which contains the variant logical type annotation. This change is hidden behind a flag that is disabled by default until read support can be properly implemented. ### Does this PR introduce _any_ user-facing change? Yes, Parquet files written by Spark 4.1.0 with the flag enabled (which it eventually will be by default) could contain the variant logical type annotation which readers without support for the type will not be able to read ### How was this patch tested? Unit test to check if nested as well as top-level variants are properly annotated, and the data is being written correctly. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53005 from harshmotw-db/harshmotw-db/variant_annotation_write. Authored-by: Harsh Motwani Signed-off-by: Wenchen Fan (cherry picked from commit 5270c997b56c3756f26bc0d771e0c4531dbbadf6) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 10 +++ .../parquet/ParquetSchemaConverter.scala | 24 +++++-- .../datasources/parquet/ParquetUtils.scala | 4 ++ .../ParquetVariantShreddingSuite.scala | 71 ++++++++++++++++++- 4 files changed, 104 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b3a251fc671e6..f3c6751a6eab7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1585,6 +1585,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE = + buildConf("spark.sql.parquet.variant.annotateLogicalType.enabled") + .doc("When enabled, Spark annotates the variant groups written to Parquet as the parquet " + + "variant logical type.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + val PARQUET_FIELD_ID_READ_ENABLED = buildConf("spark.sql.parquet.fieldId.read.enabled") .doc("Field ID is a native field of the Parquet schema spec. When enabled, Parquet readers " + @@ -7638,6 +7646,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def parquetFieldIdWriteEnabled: Boolean = getConf(SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED) + def parquetAnnotateVariantLogicalType: Boolean = getConf(PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE) + def ignoreMissingParquetFieldId: Boolean = getConf(SQLConf.IGNORE_MISSING_PARQUET_FIELD_ID) def legacyParquetNanosAsLong: Boolean = getConf(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 76e24eb03f384..d7110c7369993 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -373,6 +373,10 @@ class ParquetToSparkSchemaConverter( Option(field.getLogicalTypeAnnotation).fold( convertInternal(groupColumn, sparkReadType.map(_.asInstanceOf[StructType]))) { + // Temporary workaround to read Shredded variant data + case v: VariantLogicalTypeAnnotation if v.getSpecVersion == 1 && sparkReadType.isEmpty => + convertInternal(groupColumn, None) + // A Parquet list is represented as a 3-level structure: // // group (LIST) { @@ -552,7 +556,9 @@ class SparkToParquetSchemaConverter( writeLegacyParquetFormat: Boolean = SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get, outputTimestampType: SQLConf.ParquetOutputTimestampType.Value = SQLConf.ParquetOutputTimestampType.INT96, - useFieldId: Boolean = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.defaultValue.get) { + useFieldId: Boolean = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.defaultValue.get, + annotateVariantLogicalType: Boolean = + SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.defaultValue.get) { def this(conf: SQLConf) = this( writeLegacyParquetFormat = conf.writeLegacyParquetFormat, @@ -563,7 +569,9 @@ class SparkToParquetSchemaConverter( writeLegacyParquetFormat = conf.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key).toBoolean, outputTimestampType = SQLConf.ParquetOutputTimestampType.withName( conf.get(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key)), - useFieldId = conf.get(SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.key).toBoolean) + useFieldId = conf.get(SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.key).toBoolean, + annotateVariantLogicalType = + conf.get(SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key).toBoolean) /** * Converts a Spark SQL [[StructType]] to a Parquet [[MessageType]]. @@ -817,14 +825,22 @@ class SparkToParquetSchemaConverter( // =========== case VariantType => - Types.buildGroup(repetition) + (if (annotateVariantLogicalType) { + Types.buildGroup(repetition).as(LogicalTypeAnnotation.variantType(1)) + } else { + Types.buildGroup(repetition) + }) .addField(convertField(StructField("value", BinaryType, nullable = false), inShredded)) .addField(convertField(StructField("metadata", BinaryType, nullable = false), inShredded)) .named(field.name) case s: StructType if SparkShreddingUtils.isVariantShreddingStruct(s) => // Variant struct takes a Variant and writes to Parquet as a shredded schema. - val group = Types.buildGroup(repetition) + val group = if (annotateVariantLogicalType) { + Types.buildGroup(repetition).as(LogicalTypeAnnotation.variantType(1)) + } else { + Types.buildGroup(repetition) + } s.fields.foreach { f => group.addField(convertField(f, inShredded = true)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index 80a32711c50a7..1f11a67b08fff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -523,6 +523,10 @@ object ParquetUtils extends Logging { SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, sqlConf.legacyParquetNanosAsLong.toString) + conf.set( + SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key, + sqlConf.parquetAnnotateVariantLogicalType.toString) + // Sets compression scheme conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala index c41d88ac552da..a9ec5e161f348 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala @@ -19,11 +19,13 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File +import scala.jdk.CollectionConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetFileReader import org.apache.parquet.hadoop.util.HadoopInputFile -import org.apache.parquet.schema.{LogicalTypeAnnotation, PrimitiveType} +import org.apache.parquet.schema.{LogicalTypeAnnotation, PrimitiveType, Type} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.sql.{QueryTest, Row} @@ -154,6 +156,73 @@ class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with Share } } + test("variant logical type annotation") { + Seq(false, true).foreach { annotateVariantLogicalType => + Seq(false, true).foreach { shredVariant => + Seq(false, true).foreach { allowReadingShredded => + withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> shredVariant.toString, + SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key -> shredVariant.toString, + SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> + (allowReadingShredded || shredVariant).toString, + SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key -> + annotateVariantLogicalType.toString) { + def validateAnnotation(g: Type): Unit = { + if (annotateVariantLogicalType) { + assert(g.getLogicalTypeAnnotation == LogicalTypeAnnotation.variantType(1)) + } else { + assert(g.getLogicalTypeAnnotation == null) + } + } + withTempDir { dir => + // write parquet file + val df = spark.sql( + """ + | select + | id * 2 i, + | to_variant_object(named_struct('id', id)) v, + | named_struct('i', (id * 2)::string, + | 'nv', to_variant_object(named_struct('id', 30 + id))) ns, + | array(to_variant_object(named_struct('id', 10 + id))) av, + | map('v2', to_variant_object(named_struct('id', 20 + id))) mv + | from range(0,3,1,1)""".stripMargin) + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + val file = dir.listFiles().find(_.getName.endsWith(".parquet")).get + val parquetFilePath = file.getAbsolutePath + val inputFile = HadoopInputFile.fromPath(new Path(parquetFilePath), + new Configuration()) + val reader = ParquetFileReader.open(inputFile) + val footer = reader.getFooter + val schema = footer.getFileMetaData.getSchema + val vGroup = schema.getType(schema.getFieldIndex("v")) + validateAnnotation(vGroup) + assert(vGroup.asGroupType().getFields.asScala.toSeq + .exists(_.getName == "typed_value") == shredVariant) + val nsGroup = schema.getType(schema.getFieldIndex("ns")).asGroupType() + val nvGroup = nsGroup.getType(nsGroup.getFieldIndex("nv")) + validateAnnotation(nvGroup) + val avGroup = schema.getType(schema.getFieldIndex("av")).asGroupType() + val avList = avGroup.getType(avGroup.getFieldIndex("list")).asGroupType() + val avElement = avList.getType(avList.getFieldIndex("element")) + validateAnnotation(avElement) + val mvGroup = schema.getType(schema.getFieldIndex("mv")).asGroupType() + val mvList = mvGroup.getType(mvGroup.getFieldIndex("key_value")).asGroupType() + val mvValue = mvList.getType(mvList.getFieldIndex("value")) + validateAnnotation(mvValue) + // verify result + val result = spark.read.format("parquet") + .schema("v variant, ns struct, av array, " + + "mv map") + .load(dir.getAbsolutePath) + .selectExpr("v:id::int i1", "ns.nv:id::int i2", "av[0]:id::int i3", + "mv['v2']:id::int i4") + checkAnswer(result, Array(Row(0, 30, 10, 20), Row(1, 31, 11, 21), Row(2, 32, 12, 22))) + reader.close() + } + } + } + } + } + } testWithTempDir("write shredded variant basic") { dir => val schema = "a int, b string, c decimal(15, 1)" From d5d9ab0ca043764c13a8a5da3a54904bab26f0d0 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 12 Nov 2025 08:06:08 -0800 Subject: [PATCH 116/400] [SPARK-54320][UI] Fix Job DAG overlapping ### What changes were proposed in this pull request? Fix the Job DAG overlapping ### Why are the changes needed? A job with multiple stages might generate SVGs with overlapping ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? image ### Was this patch authored or co-authored using generative AI tooling? no Closes #53015 from yaooqinn/SPARK-54320. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit 1a802e36ed7698b7108ea7c0000efda1e6d5f949) Signed-off-by: Dongjoon Hyun --- .../apache/spark/ui/static/spark-dag-viz.js | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index fd0baec8af6c7..230c2059e6e3f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -295,6 +295,20 @@ function renderDagVizForJob(svgContainer) { .append("g") } + // Now we need to shift the container for this stage so it doesn't overlap with + // existing ones, taking into account the position and width of the last stage's + // container. We do not need to do this for the first stage of this job. + if (i > 0) { + const lastStage = svgContainer.selectAll("g.cluster.stage") + .filter((d, i, nodes) => i === nodes.length - 1); + if (lastStage) { + const lastStageWidth = toFloat(lastStage.select("rect").attr("width")); + const lastStagePosition = getAbsolutePosition(lastStage); + const offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; + container.attr("transform", `translate(${offset}, 0)`); + } + } + var g = graphlibDot.read(dot); // Actually render the stage renderDot(g, container, true); @@ -312,20 +326,6 @@ function renderDagVizForJob(svgContainer) { .attr("rx", "4") .attr("ry", "4"); - // Now we need to shift the container for this stage so it doesn't overlap with - // existing ones, taking into account the position and width of the last stage's - // container. We do not need to do this for the first stage of this job. - if (i > 0) { - var existingStages = svgContainer.selectAll("g.cluster.stage").nodes(); - if (existingStages.length > 0) { - var lastStage = d3.select(existingStages.pop()); - var lastStageWidth = toFloat(lastStage.select("rect").attr("width")); - var lastStagePosition = getAbsolutePosition(lastStage); - var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; - container.attr("transform", "translate(" + offset + ", 0)"); - } - } - // If there are any incoming edges into this graph, keep track of them to render // them separately later. Note that we cannot draw them now because we need to // put these edges in a separate container that is on top of all stage graphs. From 09aadf749b9e1d7ccf432914181550edfe2dacf6 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Wed, 12 Nov 2025 09:30:12 -0800 Subject: [PATCH 117/400] [SPARK-54172][SQL] Merge Into Schema Evolution should only add referenced columns ### What changes were proposed in this pull request? Change MERGE INTO schema evolution scope. Limit the scope of schema evolution to only add columns/nested fields that exist in source and which are directly assigned to the source column without transformation. ie, ``` UPDATE SET new_col = source.new_col UPDATE SET struct.new_field = source.struct.new_field INSERT (old_col, new_col) VALUES (s.old_col, s.new_col) ``` ### Why are the changes needed? https://github.com/apache/spark/pull/51698 added schema evolution support for MERGE INTO statements. However, it is a bit too broad. In some instances, source table may have many more fields than target tables. But user may only need a few new ones to be added to the target for the MERGE INTO statement. ### Does this PR introduce _any_ user-facing change? No, MERGE INTO schema evolution is not yet released in Spark 4.1. ### How was this patch tested? Added many unit tests in MergeIntoTableSuiteBase ### Was this patch authored or co-authored using generative AI tooling? No Closes #52866 from szehon-ho/merge_schema_evolution_limit_cols. Authored-by: Szehon Ho Signed-off-by: Dongjoon Hyun (cherry picked from commit 03eb023c3a993729e8365755d09fe88023e2be44) Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 86 +- .../analysis/ColumnResolutionHelper.scala | 15 +- .../ResolveMergeIntoSchemaEvolution.scala | 35 +- .../catalyst/plans/logical/v2Commands.scala | 133 +- .../connector/MergeIntoTableSuiteBase.scala | 1081 +++++++++++++++-- .../command/PlanResolutionSuite.scala | 36 +- 6 files changed, 1197 insertions(+), 189 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ee589d52a9d0e..f9e14cb0daf8f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1670,7 +1670,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case u: UpdateTable => resolveReferencesInUpdate(u) case m @ MergeIntoTable(targetTable, sourceTable, _, _, _, _, _) - if !m.resolved && targetTable.resolved && sourceTable.resolved && !m.needSchemaEvolution => + if !m.resolved && targetTable.resolved && sourceTable.resolved => + + // Do not throw exception for schema evolution case. + // This allows unresolved assignment keys a chance to be resolved by a second pass + // by newly column/nested fields added by schema evolution. + val throws = !m.schemaEvolutionEnabled EliminateSubqueryAliases(targetTable) match { case r: NamedRelation if r.skipSchemaResolution => @@ -1680,6 +1685,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor m case _ => + def findAttrInTarget(name: String): Option[Attribute] = { + targetTable.output.find(targetAttr => conf.resolver(name, targetAttr.name)) + } val newMatchedActions = m.matchedActions.map { case DeleteAction(deleteCondition) => val resolvedDeleteCondition = deleteCondition.map( @@ -1691,18 +1699,30 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor UpdateAction( resolvedUpdateCondition, // The update value can access columns from both target and source tables. - resolveAssignments(assignments, m, MergeResolvePolicy.BOTH)) + resolveAssignments(assignments, m, MergeResolvePolicy.BOTH, + throws = throws)) case UpdateStarAction(updateCondition) => - // Use only source columns. Missing columns in target will be handled in - // ResolveRowLevelCommandAssignments. - val assignments = targetTable.output.flatMap{ targetAttr => - sourceTable.output.find( - sourceCol => conf.resolver(sourceCol.name, targetAttr.name)) - .map(Assignment(targetAttr, _))} + // Expand star to top level source columns. If source has less columns than target, + // assignments will be added by ResolveRowLevelCommandAssignments later. + val assignments = if (m.schemaEvolutionEnabled) { + // For schema evolution case, generate assignments for missing target columns. + // These columns will be added by ResolveMergeIntoTableSchemaEvolution later. + sourceTable.output.map { sourceAttr => + val key = findAttrInTarget(sourceAttr.name).getOrElse( + UnresolvedAttribute(sourceAttr.name)) + Assignment(key, sourceAttr) + } + } else { + sourceTable.output.flatMap { sourceAttr => + findAttrInTarget(sourceAttr.name).map( + targetAttr => Assignment(targetAttr, sourceAttr)) + } + } UpdateAction( updateCondition.map(resolveExpressionByPlanChildren(_, m)), // For UPDATE *, the value must be from source table. - resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE)) + resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE, + throws = throws)) case o => o } val newNotMatchedActions = m.notMatchedActions.map { @@ -1713,21 +1733,33 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor resolveExpressionByPlanOutput(_, m.sourceTable)) InsertAction( resolvedInsertCondition, - resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE)) + resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE, + throws = throws)) case InsertStarAction(insertCondition) => // The insert action is used when not matched, so its condition and value can only // access columns from the source table. val resolvedInsertCondition = insertCondition.map( resolveExpressionByPlanOutput(_, m.sourceTable)) - // Use only source columns. Missing columns in target will be handled in - // ResolveRowLevelCommandAssignments. - val assignments = targetTable.output.flatMap{ targetAttr => - sourceTable.output.find( - sourceCol => conf.resolver(sourceCol.name, targetAttr.name)) - .map(Assignment(targetAttr, _))} + // Expand star to top level source columns. If source has less columns than target, + // assignments will be added by ResolveRowLevelCommandAssignments later. + val assignments = if (m.schemaEvolutionEnabled) { + // For schema evolution case, generate assignments for missing target columns. + // These columns will be added by ResolveMergeIntoTableSchemaEvolution later. + sourceTable.output.map { sourceAttr => + val key = findAttrInTarget(sourceAttr.name).getOrElse( + UnresolvedAttribute(sourceAttr.name)) + Assignment(key, sourceAttr) + } + } else { + sourceTable.output.flatMap { sourceAttr => + findAttrInTarget(sourceAttr.name).map( + targetAttr => Assignment(targetAttr, sourceAttr)) + } + } InsertAction( resolvedInsertCondition, - resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE)) + resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE, + throws = throws)) case o => o } val newNotMatchedBySourceActions = m.notMatchedBySourceActions.map { @@ -1741,7 +1773,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor UpdateAction( resolvedUpdateCondition, // The update value can access columns from the target table only. - resolveAssignments(assignments, m, MergeResolvePolicy.TARGET)) + resolveAssignments(assignments, m, MergeResolvePolicy.TARGET, + throws = throws)) case o => o } @@ -1818,11 +1851,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor def resolveAssignments( assignments: Seq[Assignment], mergeInto: MergeIntoTable, - resolvePolicy: MergeResolvePolicy.Value): Seq[Assignment] = { + resolvePolicy: MergeResolvePolicy.Value, + throws: Boolean): Seq[Assignment] = { assignments.map { assign => val resolvedKey = assign.key match { case c if !c.resolved => - resolveMergeExprOrFail(c, Project(Nil, mergeInto.targetTable)) + resolveMergeExpr(c, Project(Nil, mergeInto.targetTable), throws) case o => o } val resolvedValue = assign.value match { @@ -1842,7 +1876,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } else { resolvedExpr } - checkResolvedMergeExpr(withDefaultResolved, resolvePlan) + if (throws) { + checkResolvedMergeExpr(withDefaultResolved, resolvePlan) + } withDefaultResolved case o => o } @@ -1850,9 +1886,11 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } - private def resolveMergeExprOrFail(e: Expression, p: LogicalPlan): Expression = { - val resolved = resolveExprInAssignment(e, p) - checkResolvedMergeExpr(resolved, p) + private def resolveMergeExpr(e: Expression, p: LogicalPlan, throws: Boolean): Expression = { + val resolved = resolveExprInAssignment(e, p, throws) + if (throws) { + checkResolvedMergeExpr(resolved, p) + } resolved } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 53c92ca5425df..34541a8840cb9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -425,7 +425,8 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { def resolveExpressionByPlanChildren( e: Expression, q: LogicalPlan, - includeLastResort: Boolean = false): Expression = { + includeLastResort: Boolean = false, + throws: Boolean = true): Expression = { resolveExpression( tryResolveDataFrameColumns(e, q.children), resolveColumnByName = nameParts => { @@ -435,7 +436,7 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { assert(q.children.length == 1) q.children.head.output }, - throws = true, + throws, includeLastResort = includeLastResort) } @@ -475,8 +476,14 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { resolveVariables(resolveOuterRef(e)) } - def resolveExprInAssignment(expr: Expression, hostPlan: LogicalPlan): Expression = { - resolveExpressionByPlanChildren(expr, hostPlan) match { + def resolveExprInAssignment( + expr: Expression, + hostPlan: LogicalPlan, + throws: Boolean = true): Expression = { + resolveExpressionByPlanChildren(expr, + hostPlan, + includeLastResort = false, + throws = throws) match { // Assignment key and value does not need the alias when resolving nested columns. case Alias(child: ExtractValue, _) => child case other => other diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala index 7e7776098a04a..f317a2efddbe4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsRowLevelOperations, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.StructType /** @@ -34,24 +35,38 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation object ResolveMergeIntoSchemaEvolution extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case m @ MergeIntoTable(_, _, _, _, _, _, _) - if m.needSchemaEvolution => - val newTarget = m.targetTable.transform { - case r : DataSourceV2Relation => performSchemaEvolution(r, m.sourceTable) + // This rule should run only if all assignments are resolved, except those + // that will be satisfied by schema evolution + case m@MergeIntoTable(_, _, _, _, _, _, _) if m.evaluateSchemaEvolution => + val changes = m.changesForSchemaEvolution + if (changes.isEmpty) { + m + } else { + m transformUpWithNewOutput { + case r @ DataSourceV2Relation(_: SupportsRowLevelOperations, _, _, _, _, _) => + val referencedSourceSchema = MergeIntoTable.sourceSchemaForSchemaEvolution(m) + val newTarget = performSchemaEvolution(r, referencedSourceSchema, changes) + val oldTargetOutput = m.targetTable.output + val newTargetOutput = newTarget.output + val attributeMapping = oldTargetOutput.map( + oldAttr => (oldAttr, newTargetOutput.find(_.name == oldAttr.name).getOrElse(oldAttr)) + ) + newTarget -> attributeMapping } - m.copy(targetTable = newTarget) + } } - private def performSchemaEvolution(relation: DataSourceV2Relation, source: LogicalPlan) - : DataSourceV2Relation = { + private def performSchemaEvolution( + relation: DataSourceV2Relation, + referencedSourceSchema: StructType, + changes: Array[TableChange]): DataSourceV2Relation = { (relation.catalog, relation.identifier) match { case (Some(c: TableCatalog), Some(i)) => - val changes = MergeIntoTable.schemaChanges(relation.schema, source.schema) c.alterTable(i, changes: _*) val newTable = c.loadTable(i) val newSchema = CatalogV2Util.v2ColumnsToStructType(newTable.columns()) // Check if there are any remaining changes not applied. - val remainingChanges = MergeIntoTable.schemaChanges(newSchema, source.schema) + val remainingChanges = MergeIntoTable.schemaChanges(newSchema, referencedSourceSchema) if (remainingChanges.nonEmpty) { throw QueryCompilationErrors.unsupportedTableChangesInAutoSchemaEvolutionError( remainingChanges, i.toQualifiedNameParts(c)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index cd0c2742df3d5..db5dacdcef38a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.{SparkIllegalArgumentException, SparkUnsupportedOperationException} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AssignmentUtils, EliminateSubqueryAliases, FieldName, NamedRelation, PartitionSpec, ResolvedIdentifier, ResolvedProcedure, TypeCheckResult, UnresolvedException, UnresolvedProcedure, ViewSchemaMode} +import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AssignmentUtils, EliminateSubqueryAliases, FieldName, NamedRelation, PartitionSpec, ResolvedIdentifier, ResolvedProcedure, TypeCheckResult, UnresolvedAttribute, UnresolvedException, UnresolvedProcedure, ViewSchemaMode} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} import org.apache.spark.sql.catalyst.catalog.{FunctionResource, RoutineLanguage} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -893,16 +893,46 @@ case class MergeIntoTable( } lazy val needSchemaEvolution: Boolean = + evaluateSchemaEvolution && changesForSchemaEvolution.nonEmpty + + lazy val evaluateSchemaEvolution: Boolean = schemaEvolutionEnabled && - MergeIntoTable.schemaChanges(targetTable.schema, sourceTable.schema).nonEmpty + canEvaluateSchemaEvolution - private def schemaEvolutionEnabled: Boolean = withSchemaEvolution && { + lazy val schemaEvolutionEnabled: Boolean = withSchemaEvolution && { EliminateSubqueryAliases(targetTable) match { case r: DataSourceV2Relation if r.autoSchemaEvolution() => true case _ => false } } + // Guard that assignments are either resolved or candidates for evolution before + // evaluating schema evolution. We need to use resolved assignment values to check + // candidates, see MergeIntoTable.sourceSchemaForSchemaEvolution for details. + lazy val canEvaluateSchemaEvolution: Boolean = { + if ((!targetTable.resolved) || (!sourceTable.resolved)) { + false + } else { + val actions = matchedActions ++ notMatchedActions + val assignments = actions.collect { + case a: UpdateAction => a.assignments + case a: InsertAction => a.assignments + }.flatten + + val sourcePaths = MergeIntoTable.extractAllFieldPaths(sourceTable.schema) + assignments.forall { assignment => + assignment.resolved || + sourcePaths.exists { path => MergeIntoTable.isEqual(assignment, path) } + } + } + } + + private lazy val sourceSchemaForEvolution: StructType = + MergeIntoTable.sourceSchemaForSchemaEvolution(this) + + lazy val changesForSchemaEvolution: Array[TableChange] = + MergeIntoTable.schemaChanges(targetTable.schema, sourceSchemaForEvolution) + override def left: LogicalPlan = targetTable override def right: LogicalPlan = sourceTable override protected def withNewChildrenInternal( @@ -911,6 +941,7 @@ case class MergeIntoTable( } object MergeIntoTable { + def getWritePrivileges( matchedActions: Iterable[MergeAction], notMatchedActions: Iterable[MergeAction], @@ -948,11 +979,12 @@ object MergeIntoTable { case currentField: StructField if newFieldMap.contains(currentField.name) => schemaChanges(currentField.dataType, newFieldMap(currentField.name).dataType, originalTarget, originalSource, fieldPath ++ Seq(currentField.name)) - }}.flatten + } + }.flatten // Identify the newly added fields and append to the end val currentFieldMap = toFieldMap(currentFields) - val adds = newFields.filterNot (f => currentFieldMap.contains (f.name)) + val adds = newFields.filterNot(f => currentFieldMap.contains(f.name)) .map(f => TableChange.addColumn(fieldPath ++ Set(f.name), f.dataType)) updates ++ adds @@ -990,6 +1022,97 @@ object MergeIntoTable { CaseInsensitiveMap(fieldMap) } } + + // A pruned version of source schema that only contains columns/nested fields + // explicitly and directly assigned to a target counterpart in MERGE INTO actions, + // which are relevant for schema evolution. + // Examples: + // * UPDATE SET target.a = source.a + // * UPDATE SET nested.a = source.nested.a + // * INSERT (a, nested.b) VALUES (source.a, source.nested.b) + // New columns/nested fields in this schema that are not existing in target schema + // will be added for schema evolution. + def sourceSchemaForSchemaEvolution(merge: MergeIntoTable): StructType = { + val actions = merge.matchedActions ++ merge.notMatchedActions + val assignments = actions.collect { + case a: UpdateAction => a.assignments + case a: InsertAction => a.assignments + }.flatten + + val containsStarAction = actions.exists { + case _: UpdateStarAction => true + case _: InsertStarAction => true + case _ => false + } + + def filterSchema(sourceSchema: StructType, basePath: Seq[String]): StructType = + StructType(sourceSchema.flatMap { field => + val fieldPath = basePath :+ field.name + + field.dataType match { + // Specifically assigned to in one clause: + // always keep, including all nested attributes + case _ if assignments.exists(isEqual(_, fieldPath)) => Some(field) + // If this is a struct and one of the children is being assigned to in a merge clause, + // keep it and continue filtering children. + case struct: StructType if assignments.exists(assign => + isPrefix(fieldPath, extractFieldPath(assign.key, allowUnresolved = true))) => + Some(field.copy(dataType = filterSchema(struct, fieldPath))) + // The field isn't assigned to directly or indirectly (i.e. its children) in any non-* + // clause. Check if it should be kept with any * action. + case struct: StructType if containsStarAction => + Some(field.copy(dataType = filterSchema(struct, fieldPath))) + case _ if containsStarAction => Some(field) + // The field and its children are not assigned to in any * or non-* action, drop it. + case _ => None + } + }) + + filterSchema(merge.sourceTable.schema, Seq.empty) + } + + private def extractAllFieldPaths(schema: StructType, basePath: Seq[String] = Seq.empty): + Seq[Seq[String]] = { + schema.flatMap { field => + val fieldPath = basePath :+ field.name + field.dataType match { + case struct: StructType => + fieldPath +: extractAllFieldPaths(struct, fieldPath) + case _ => + Seq(fieldPath) + } + } + } + + // Helper method to extract field path from an Expression. + private def extractFieldPath(expr: Expression, allowUnresolved: Boolean): Seq[String] = { + expr match { + case UnresolvedAttribute(nameParts) if allowUnresolved => nameParts + case a: AttributeReference => Seq(a.name) + case GetStructField(child, ordinal, nameOpt) => + extractFieldPath(child, allowUnresolved) :+ nameOpt.getOrElse(s"col$ordinal") + case _ => Seq.empty + } + } + + // Helper method to check if a given field path is a prefix of another path. + private def isPrefix(prefix: Seq[String], path: Seq[String]): Boolean = + prefix.length <= path.length && prefix.zip(path).forall { + case (prefixNamePart, pathNamePart) => + SQLConf.get.resolver(prefixNamePart, pathNamePart) + } + + // Helper method to check if an assignment key is equal to a source column + // and if the assignment value is that same source column. + // Example: UPDATE SET target.a = source.a + private def isEqual(assignment: Assignment, sourceFieldPath: Seq[String]): Boolean = { + // key must be a non-qualified field path that may be added to target schema via evolution + val assignmenKeyExpr = extractFieldPath(assignment.key, allowUnresolved = true) + // value should always be resolved (from source) + val assignmentValueExpr = extractFieldPath(assignment.value, allowUnresolved = false) + assignmenKeyExpr == assignmentValueExpr && + assignmenKeyExpr == sourceFieldPath + } } sealed abstract class MergeAction extends Expression with Unevaluable { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index 98706c4afeae9..b73c8d2458c87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -2206,6 +2206,118 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } + test("Merge schema evolution new column with conditions on update and insert") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |{ "pk": 4, "salary": 400, "dep": "marketing" } + |{ "pk": 5, "salary": 500, "dep": "executive" } + |""".stripMargin) + + // Two rows that could be updated (pk 4 and 5), but only one has salary > 450 + // Two rows that could be inserted (pk 6 and 7), but only one has active = true + val sourceDF = Seq((4, 450, "finance", false), + (5, 550, "finance", true), + (6, 350, "sales", true), + (7, 250, "sales", false)).toDF("pk", "salary", "dep", "active") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED AND s.salary > 450 THEN + | UPDATE SET dep='updated', active=s.active + |WHEN NOT MATCHED AND s.active = true THEN + | INSERT (pk, salary, dep, active) VALUES (s.pk, s.salary, s.dep, + | s.active) + |""".stripMargin + + if (withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr", null), + Row(2, 200, "software", null), + Row(3, 300, "hr", null), + Row(4, 400, "marketing", null), // pk=4 not updated (salary 450 is not > 450) + Row(5, 500, "updated", true), // pk=5 updated (salary 550 > 450) + Row(6, 350, "sales", true))) // pk=6 inserted (active = true) + // pk=7 not inserted (active = false) + } else { + val e = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(e.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(e.getMessage.contains("A column, variable, or function parameter with name " + + "`active` cannot be resolved")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution with condition on new column from target") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |{ "pk": 4, "salary": 400, "dep": "marketing" } + |{ "pk": 5, "salary": 500, "dep": "executive" } + |""".stripMargin) + + // Source has new 'active' column that doesn't exist in target + val sourceDF = Seq((4, 450, "finance", true), + (5, 550, "finance", false), + (6, 350, "sales", true)).toDF("pk", "salary", "dep", "active") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + // Condition references t.active which doesn't exist yet in target + val mergeStmt = s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED AND t.active IS NULL THEN + | UPDATE SET salary=s.salary, dep=s.dep, active=s.active + |WHEN NOT MATCHED THEN + | INSERT (pk, salary, dep, active) + | VALUES (s.pk, s.salary, s.dep, s.active) + |""".stripMargin + + if (withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr", null), + Row(2, 200, "software", null), + Row(3, 300, "hr", null), + Row(4, 450, "finance", true), // Updated (t.active was NULL) + Row(5, 550, "finance", false), // Updated (t.active was NULL) + Row(6, 350, "sales", true))) // Inserted + } else { + val e = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(e.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(e.getMessage.contains("A column, variable, or function parameter with name " + + "`active` cannot be resolved")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + test("Merge schema evolution new column with set all columns") { Seq((true, true), (false, true), (true, false)).foreach { case (withSchemaEvolution, schemaEvolutionEnabled) => @@ -3510,155 +3622,800 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } - test("merge into with source missing fields in top-level struct") { - withTempView("source") { - // Target table has struct with 3 fields at top level - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT, - |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": true }, "dep": "sales"}""") - - // Source table has struct with only 2 fields (c1, c2) - missing c3 - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StringType)))), // missing c3 field - StructField("dep", StringType))) - val data = Seq( - Row(1, Row(10, "b"), "hr"), - Row(2, Row(20, "c"), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") - - sql( - s"""MERGE INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) + test("Merge schema evolution should not evolve referencing new column via transform") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) - // Missing field c3 should be filled with NULL - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, "a", true), "sales"), - Row(1, Row(10, "b", null), "hr"), - Row(2, Row(20, "c", null), "engineering"))) - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") - } + val sourceDF = Seq((2, 150, "dummy", "blah"), + (3, 250, "dummy", "blah")).toDF("pk", "salary", "dep", "extra") + sourceDF.createOrReplaceTempView("source") - test("merge into with source missing fields in struct nested in array") { - withTempView("source") { - // Target table has struct with 3 fields (c1, c2, c3) in array - createAndInitTable( - s"""pk INT NOT NULL, - |a ARRAY>, - |dep STRING""".stripMargin, - """{ "pk": 0, "a": [ { "c1": 1, "c2": "a", "c3": true } ], "dep": "sales" } - |{ "pk": 1, "a": [ { "c1": 2, "c2": "b", "c3": false } ], "dep": "sales" }""" - .stripMargin) + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET extra=substring(s.extra, 1, 2) + |""".stripMargin - // Source table has struct with only 2 fields (c1, c2) - missing c3 - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("a", ArrayType( - StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StringType))))), // missing c3 field - StructField("dep", StringType))) - val data = Seq( - Row(1, Array(Row(10, "c")), "hr"), - Row(2, Array(Row(30, "e")), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") - sql( - s"""MERGE INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) + val e = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(e.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(e.getMessage.contains("A column, variable, or function parameter with name " + + "`extra` cannot be resolved")) - // Missing field c3 should be filled with NULL - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Array(Row(1, "a", true)), "sales"), - Row(1, Array(Row(10, "c", null)), "hr"), - Row(2, Array(Row(30, "e", null)), "engineering"))) + sql(s"DROP TABLE $tableNameAsString") + } } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - test("merge into with source missing fields in struct nested in map key") { - withTempView("source") { - // Target table has struct with 2 fields in map key - val targetSchema = - StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("m", MapType( - StructType(Seq(StructField("c1", IntegerType), StructField("c2", BooleanType))), - StructType(Seq(StructField("c3", StringType))))), - StructField("dep", StringType))) - createTable(CatalogV2Util.structTypeToV2Columns(targetSchema)) + test("Merge schema evolution should not evolve if not directly referencing new column: update") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) - val targetData = Seq( - Row(0, Map(Row(10, true) -> Row("x")), "hr"), - Row(1, Map(Row(20, false) -> Row("y")), "sales")) - spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) - .writeTo(tableNameAsString).append() + val sourceDF = Seq((2, 150, "dummy", "blah"), + (3, 250, "dummy", "blah")).toDF("pk", "salary", "dep", "extra") + sourceDF.createOrReplaceTempView("source") - // Source table has struct with only 1 field (c1) in map key - missing c2 - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("m", MapType( - StructType(Seq(StructField("c1", IntegerType))), // missing c2 - StructType(Seq(StructField("c3", StringType))))), - StructField("dep", StringType))) - val sourceData = Seq( - Row(1, Map(Row(10) -> Row("z")), "sales"), - Row(2, Map(Row(20) -> Row("w")), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) - .createOrReplaceTempView("source") + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET dep='software' + |""".stripMargin - sql( - s"""MERGE INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr"), + Row(2, 200, "software"))) - // Missing field c2 should be filled with NULL - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Map(Row(10, true) -> Row("x")), "hr"), - Row(1, Map(Row(10, null) -> Row("z")), "sales"), - Row(2, Map(Row(20, null) -> Row("w")), "engineering"))) + sql(s"DROP TABLE $tableNameAsString") + } } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - test("merge into with source missing fields in struct nested in map value") { - withTempView("source") { - // Target table has struct with 2 fields in map value - val targetSchema = - StructType(Seq( + test("Merge schema evolution should not evolve if not directly referencing new column: insert") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceDF = Seq((2, 150, "dummy", "blah"), + (3, 250, "dummy", "blah")).toDF("pk", "salary", "dep", "extra") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN NOT MATCHED THEN + | INSERT (pk, salary, dep) VALUES (s.pk, s.salary, 'newdep') + |""".stripMargin + + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr"), + Row(2, 200, "software"), + Row(3, 250, "newdep"))) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should not evolve if not directly referencing new column:" + + "update and insert") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceDF = Seq((2, 150, "dummy", "blah"), + (3, 250, "dummy", "blah")).toDF("pk", "salary", "dep", "extra") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET dep='software' + |WHEN NOT MATCHED THEN + | INSERT (pk, salary, dep) VALUES (s.pk, s.salary, 'newdep') + |""".stripMargin + + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr"), + Row(2, 200, "software"), + Row(3, 250, "newdep"))) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should not evolve if not having just column name: update") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceDF = Seq((2, 150, "dummy", "blah"), + (3, 250, "dummy", "blah")).toDF("pk", "salary", "dep", "extra") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET t.extra = s.extra + |""".stripMargin + + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(exception.message.contains(" A column, variable, or function parameter with name " + + "`t`.`extra` cannot be resolved")) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should only evolve referenced column when source " + + "has multiple new columns") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceDF = Seq((2, 150, "dummy", 50, "blah"), + (3, 250, "dummy", 75, "blah")).toDF("pk", "salary", "dep", "bonus", "extra") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET salary = s.salary, bonus = s.bonus + |WHEN NOT MATCHED THEN + | INSERT (pk, salary, dep, bonus) VALUES (s.pk, s.salary, 'newdep', s.bonus) + |""".stripMargin + + if (withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr", null), + Row(2, 150, "software", 50), + Row(3, 250, "newdep", 75))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should only evolve referenced struct field when source " + + "has multiple new struct fields") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |info STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 1, "info": { "salary": 100, "status": "active" }, "dep": "hr" } + |{ "pk": 2, "info": { "salary": 200, "status": "inactive" }, "dep": "software" } + |""".stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("info", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType), + StructField("bonus", IntegerType), // new field 1 + StructField("extra", StringType) // new field 2 + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(2, Row(150, "dummy", 50, "blah"), "active"), + Row(3, Row(250, "dummy", 75, "blah"), "active") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET info.bonus = s.info.bonus + |""".stripMargin + + if (withSchemaEvolution) { + sql(mergeStmt) + // Only 'bonus' field should be added, not 'extra' + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(100, "active", null), "hr"), + Row(2, Row(200, "inactive", 50), "software"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "FIELD_NOT_FOUND") + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should not evolve when assigning existing target column " + + "from source column that does not exist in target") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceDF = Seq((2, 150, "dummy", 50), + (3, 250, "dummy", 75)).toDF("pk", "salary", "dep", "bonus") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET salary = s.bonus + |WHEN NOT MATCHED THEN + | INSERT (pk, salary, dep) VALUES (s.pk, s.bonus, 'newdep') + |""".stripMargin + + sql(mergeStmt) + // bonus column should NOT be added to target schema + // Only salary is updated with bonus value + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr"), + Row(2, 50, "software"), + Row(3, 75, "newdep"))) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should not evolve struct if not directly referencing new field " + + "in top level struct: insert") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |info STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 1, "info": { "salary": 100, "status": "active" }, "dep": "hr" } + |{ "pk": 2, "info": { "salary": 200, "status": "inactive" }, "dep": "software" } + |""".stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("info", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType), + StructField("bonus", IntegerType) // new field not in target + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(2, Row(150, "dummy", 50), "active"), + Row(3, Row(250, "dummy", 75), "active") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN NOT MATCHED THEN + | INSERT (pk, info, dep) VALUES (s.pk, + | named_struct('salary', s.info.salary, 'status', 'active'), 'marketing') + |""".stripMargin + + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(100, "active"), "hr"), + Row(2, Row(200, "inactive"), "software"), + Row(3, Row(250, "active"), "marketing"))) + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should not evolve if not directly referencing new field " + + "in top level struct: UPDATE") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |info STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 1, "info": { "salary": 100, "status": "active" }, "dep": "hr" } + |{ "pk": 2, "info": { "salary": 200, "status": "inactive" }, "dep": "software" } + |""".stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("info", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType), + StructField("bonus", IntegerType) // new field not in target + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(2, Row(150, "dummy", 50), "active"), + Row(3, Row(250, "dummy", 75), "active") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET info.status='inactive' + |""".stripMargin + + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(100, "active"), "hr"), + Row(2, Row(200, "inactive"), "software"))) + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should evolve when directly assigning struct with new field:" + + "UPDATE") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |info STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 1, "info": { "salary": 100, "status": "active" }, "dep": "hr" } + |{ "pk": 2, "info": { "salary": 200, "status": "inactive" }, "dep": "software" } + |""".stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("info", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType), + StructField("bonus", IntegerType) // new field not in target + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(2, Row(150, "updated", 50), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET info = s.info + |""".stripMargin + + if (withSchemaEvolution) { + sql(mergeStmt) + // Schema should evolve - bonus field should be added + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(100, "active", null), "hr"), + Row(2, Row(150, "updated", 50), "software"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.getMessage.contains("Cannot safely cast") || + exception.getMessage.contains("incompatible")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should evolve when directly assigning struct with new field: " + + "INSERT") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |info STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 1, "info": { "salary": 100, "status": "active" }, "dep": "hr" } + |{ "pk": 2, "info": { "salary": 200, "status": "inactive" }, "dep": "software" } + |""".stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("info", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType), + StructField("bonus", IntegerType) // new field not in target + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(3, Row(150, "new", 50), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN NOT MATCHED THEN + | INSERT (pk, info, dep) VALUES (s.pk, s.info, s.dep) + |""".stripMargin + + if (withSchemaEvolution) { + sql(mergeStmt) + // Schema should evolve - bonus field should be added + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(100, "active", null), "hr"), + Row(2, Row(200, "inactive", null), "software"), + Row(3, Row(150, "new", 50), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.getMessage.contains("Cannot safely cast") || + exception.getMessage.contains("incompatible")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should not evolve if not directly referencing " + + "new field in nested struct") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("employee", StructType(Seq( + StructField("name", StringType), + StructField("details", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType) + ))) + ))), + StructField("dep", StringType) + )) + + createTable(CatalogV2Util.structTypeToV2Columns(targetSchema)) + + val targetData = Seq( + Row(1, Row("Alice", Row(100, "active")), "hr"), + Row(2, Row("Bob", Row(200, "active")), "software") + ) + spark.createDataFrame( + spark.sparkContext.parallelize(targetData), targetSchema) + .coalesce(1).writeTo(tableNameAsString).append() + + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row("Alice", Row(100, "active")), "hr"), + Row(2, Row("Bob", Row(200, "active")), "software"))) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("employee", StructType(Seq( + StructField("name", StringType), + StructField("details", StructType(Seq( + StructField("salary", IntegerType), + StructField("status", StringType), + StructField("bonus", IntegerType) // new field not in target + ))) + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(2, Row("Bob", Row(150, "active", 50)), "dummy"), + Row(3, Row("Charlie", Row(250, "active", 75)), "dummy") + ) + spark.createDataFrame( + spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = + if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET employee.details.status='inactive' + |""".stripMargin + + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row("Alice", Row(100, "active")), "hr"), + Row(2, Row("Bob", Row(200, "inactive")), "software"))) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("Merge schema evolution should evolve referencing new column assigned to something else") { + Seq(true, false).foreach { withSchemaEvolution => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceDF = Seq((2, 150, "dummy", "blah"), + (3, 250, "dummy", "blah")).toDF("pk", "salary", "dep", "extra") + sourceDF.createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET extra=s.dep + |""".stripMargin + + val e = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(e.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(e.getMessage.contains("A column, variable, or function parameter with name " + + "`extra` cannot be resolved")) + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge into with source missing fields in top-level struct") { + withTempView("source") { + // Target table has struct with 3 fields at top level + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": true }, "dep": "sales"}""") + + // Source table has struct with only 2 fields (c1, c2) - missing c3 + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType)))), // missing c3 field + StructField("dep", StringType))) + val data = Seq( + Row(1, Row(10, "b"), "hr"), + Row(2, Row(20, "c"), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + + // Missing field c3 should be filled with NULL + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a", true), "sales"), + Row(1, Row(10, "b", null), "hr"), + Row(2, Row(20, "c", null), "engineering"))) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge into with source missing fields in struct nested in array") { + withTempView("source") { + // Target table has struct with 3 fields (c1, c2, c3) in array + createAndInitTable( + s"""pk INT NOT NULL, + |a ARRAY>, + |dep STRING""".stripMargin, + """{ "pk": 0, "a": [ { "c1": 1, "c2": "a", "c3": true } ], "dep": "sales" } + |{ "pk": 1, "a": [ { "c1": 2, "c2": "b", "c3": false } ], "dep": "sales" }""" + .stripMargin) + + // Source table has struct with only 2 fields (c1, c2) - missing c3 + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("a", ArrayType( + StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType))))), // missing c3 field + StructField("dep", StringType))) + val data = Seq( + Row(1, Array(Row(10, "c")), "hr"), + Row(2, Array(Row(30, "e")), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + + // Missing field c3 should be filled with NULL + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Array(Row(1, "a", true)), "sales"), + Row(1, Array(Row(10, "c", null)), "hr"), + Row(2, Array(Row(30, "e", null)), "engineering"))) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge into with source missing fields in struct nested in map key") { + withTempView("source") { + // Target table has struct with 2 fields in map key + val targetSchema = + StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("m", MapType( + StructType(Seq(StructField("c1", IntegerType), StructField("c2", BooleanType))), + StructType(Seq(StructField("c3", StringType))))), + StructField("dep", StringType))) + createTable(CatalogV2Util.structTypeToV2Columns(targetSchema)) + + val targetData = Seq( + Row(0, Map(Row(10, true) -> Row("x")), "hr"), + Row(1, Map(Row(20, false) -> Row("y")), "sales")) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + // Source table has struct with only 1 field (c1) in map key - missing c2 + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("m", MapType( + StructType(Seq(StructField("c1", IntegerType))), // missing c2 + StructType(Seq(StructField("c3", StringType))))), + StructField("dep", StringType))) + val sourceData = Seq( + Row(1, Map(Row(10) -> Row("z")), "sales"), + Row(2, Map(Row(20) -> Row("w")), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + + // Missing field c2 should be filled with NULL + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Map(Row(10, true) -> Row("x")), "hr"), + Row(1, Map(Row(10, null) -> Row("z")), "sales"), + Row(2, Map(Row(20, null) -> Row("w")), "engineering"))) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge into with source missing fields in struct nested in map value") { + withTempView("source") { + // Target table has struct with 2 fields in map value + val targetSchema = + StructType(Seq( StructField("pk", IntegerType, nullable = false), StructField("m", MapType( StructType(Seq(StructField("c1", IntegerType))), @@ -3820,6 +4577,62 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"DROP TABLE IF EXISTS $tableNameAsString") } + test("Merge schema evolution should error on non-existent column in UPDATE and INSERT") { + withTable(tableNameAsString) { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |salary INT, + |dep STRING""".stripMargin, + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("salary", IntegerType), + StructField("dep", StringType) + )) + + val data = Seq( + Row(2, 250, "engineering"), + Row(3, 300, "finance") + ) + + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val updateException = intercept[AnalysisException] { + sql( + s"""MERGE WITH SCHEMA EVOLUTION + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET non_existent = s.nonexistent_column + |""".stripMargin) + } + assert(updateException.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(updateException.message.contains("A column, variable, or function parameter " + + "with name `non_existent` cannot be resolved")) + + val insertException = intercept[AnalysisException] { + sql( + s"""MERGE WITH SCHEMA EVOLUTION + |INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN NOT MATCHED THEN + | INSERT (pk, salary, dep, non_existent) VALUES (s.pk, s.salary, s.dep, s.dep) + |""".stripMargin) + } + assert(insertException.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(insertException.message.contains("A column, variable, or function parameter " + + "with name `non_existent` cannot be resolved")) + } + } + } + private def findMergeExec(query: String): MergeRowsExec = { val plan = executeAndKeepPlan { sql(query) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 9ea8b9130ba8a..dfd24a1ebe970 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -1633,10 +1633,10 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { if (starInUpdate) { assert(updateAssigns.size == 2) - assert(updateAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ti)) - assert(updateAssigns(0).value.asInstanceOf[AttributeReference].sameRef(si)) - assert(updateAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ts)) - assert(updateAssigns(1).value.asInstanceOf[AttributeReference].sameRef(ss)) + assert(updateAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(updateAssigns(0).value.asInstanceOf[AttributeReference].sameRef(ss)) + assert(updateAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ti)) + assert(updateAssigns(1).value.asInstanceOf[AttributeReference].sameRef(si)) } else { assert(updateAssigns.size == 1) assert(updateAssigns.head.key.asInstanceOf[AttributeReference].sameRef(ts)) @@ -1648,15 +1648,25 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { target: LogicalPlan, source: LogicalPlan, insertCondAttr: Option[AttributeReference], - insertAssigns: Seq[Assignment]): Unit = { + insertAssigns: Seq[Assignment], + starInInsert: Boolean = false): Unit = { val (si, ss) = getAttributes(source) val (ti, ts) = getAttributes(target) insertCondAttr.foreach(a => assert(a.sameRef(ss))) - assert(insertAssigns.size == 2) - assert(insertAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ti)) - assert(insertAssigns(0).value.asInstanceOf[AttributeReference].sameRef(si)) - assert(insertAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ts)) - assert(insertAssigns(1).value.asInstanceOf[AttributeReference].sameRef(ss)) + + if (starInInsert) { + assert(insertAssigns.size == 2) + assert(insertAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(insertAssigns(0).value.asInstanceOf[AttributeReference].sameRef(ss)) + assert(insertAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ti)) + assert(insertAssigns(1).value.asInstanceOf[AttributeReference].sameRef(si)) + } else { + assert(insertAssigns.size == 2) + assert(insertAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ti)) + assert(insertAssigns(0).value.asInstanceOf[AttributeReference].sameRef(si)) + assert(insertAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(insertAssigns(1).value.asInstanceOf[AttributeReference].sameRef(ss)) + } } def checkNotMatchedBySourceClausesResolution( @@ -1735,7 +1745,8 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { checkMergeConditionResolution(target, source, mergeCondition) checkMatchedClausesResolution(target, source, Some(dl), Some(ul), updateAssigns, starInUpdate = true) - checkNotMatchedClausesResolution(target, source, Some(il), insertAssigns) + checkNotMatchedClausesResolution(target, source, Some(il), insertAssigns, + starInInsert = true) assert(withSchemaEvolution === false) case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) @@ -1762,7 +1773,8 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { checkMergeConditionResolution(target, source, mergeCondition) checkMatchedClausesResolution(target, source, None, None, updateAssigns, starInUpdate = true) - checkNotMatchedClausesResolution(target, source, None, insertAssigns) + checkNotMatchedClausesResolution(target, source, None, insertAssigns, + starInInsert = true) assert(withSchemaEvolution === false) case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) From cd1ff2a33cd6d20ceef4de7ef17d0d9aa604530f Mon Sep 17 00:00:00 2001 From: vinodkc Date: Wed, 12 Nov 2025 09:42:57 -0800 Subject: [PATCH 118/400] [SPARK-54206][CONNECT] Support BINARY type data in SparkConnectResultSet ### What changes were proposed in this pull request? Add BINARY type support to the Spark Connect JDBC client ### Why are the changes needed? BINARY is a fundamental SQL type required for JDBC compliance and interoperability. ### Does this PR introduce _any_ user-facing change? Yes, it's part of a new feature under Spark connect JDBC support. ### How was this patch tested? Added new UTs in `SparkConnectJdbcDataTypeSuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #53018 from vinodkc/br_SPARK-54206. Authored-by: vinodkc Signed-off-by: Dongjoon Hyun (cherry picked from commit ecaec3d1b013081900c6cab789315c034ae2201d) Signed-off-by: Dongjoon Hyun --- .../client/jdbc/SparkConnectResultSet.scala | 9 ++- .../client/jdbc/util/JdbcTypeUtils.scala | 8 +- .../jdbc/SparkConnectJdbcDataTypeSuite.scala | 78 ++++++++++++++++++- 3 files changed, 87 insertions(+), 8 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index 5544dee6df5f6..5fd425027173a 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -137,8 +137,11 @@ class SparkConnectResultSet( override def getBigDecimal(columnIndex: Int, scale: Int): java.math.BigDecimal = throw new SQLFeatureNotSupportedException - override def getBytes(columnIndex: Int): Array[Byte] = - throw new SQLFeatureNotSupportedException + override def getBytes(columnIndex: Int): Array[Byte] = { + getColumnValue(columnIndex, null: Array[Byte]) { idx => + currentRow.get(idx).asInstanceOf[Array[Byte]] + } + } override def getDate(columnIndex: Int): Date = { getColumnValue(columnIndex, null: Date) { idx => currentRow.getDate(idx) } @@ -187,7 +190,7 @@ class SparkConnectResultSet( throw new SQLFeatureNotSupportedException override def getBytes(columnLabel: String): Array[Byte] = - throw new SQLFeatureNotSupportedException + getBytes(findColumn(columnLabel)) override def getDate(columnLabel: String): Date = getDate(findColumn(columnLabel)) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala index 4af67f1582d32..f1cb1236578cc 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala @@ -37,6 +37,7 @@ private[jdbc] object JdbcTypeUtils { case StringType => Types.VARCHAR case _: DecimalType => Types.DECIMAL case DateType => Types.DATE + case BinaryType => Types.BINARY case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -53,6 +54,7 @@ private[jdbc] object JdbcTypeUtils { case StringType => classOf[String].getName case _: DecimalType => classOf[JBigDecimal].getName case DateType => classOf[Date].getName + case BinaryType => classOf[Array[Byte]].getName case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -60,7 +62,7 @@ private[jdbc] object JdbcTypeUtils { def isSigned(field: StructField): Boolean = field.dataType match { case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | _: DecimalType => true - case NullType | BooleanType | StringType | DateType => false + case NullType | BooleanType | StringType | DateType | BinaryType => false case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -77,6 +79,7 @@ private[jdbc] object JdbcTypeUtils { case StringType => 255 case DecimalType.Fixed(p, _) => p case DateType => 10 + case BinaryType => Int.MaxValue case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -85,7 +88,7 @@ private[jdbc] object JdbcTypeUtils { case FloatType => 7 case DoubleType => 15 case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | StringType | - DateType => 0 + DateType | BinaryType => 0 case DecimalType.Fixed(_, s) => s case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") @@ -101,6 +104,7 @@ private[jdbc] object JdbcTypeUtils { case StringType => getPrecision(field) case DateType => 10 // length of `YYYY-MM-DD` + case BinaryType => Int.MaxValue // precision + negative sign + leading zero + decimal point, like DECIMAL(5,5) = -0.12345 case DecimalType.Fixed(p, s) if p == s => p + 3 // precision + negative sign, like DECIMAL(5,0) = -12345 diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index 1820f30732daf..e06e3474e07b0 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -259,7 +259,8 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess ("cast(1 AS BIGINT)", (rs: ResultSet) => rs.getLong(999)), ("cast(1 AS FLOAT)", (rs: ResultSet) => rs.getFloat(999)), ("cast(1 AS DOUBLE)", (rs: ResultSet) => rs.getDouble(999)), - ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(999)) + ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(999)), + ("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(999)) ).foreach { case (query, getter) => withExecuteQuery(s"SELECT $query") { rs => @@ -284,13 +285,18 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess ("cast(1 AS FLOAT)", (rs: ResultSet) => rs.getFloat(1), 1.toFloat), ("cast(1 AS DOUBLE)", (rs: ResultSet) => rs.getDouble(1), 1.toDouble), ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(1), - new java.math.BigDecimal("1.00000")) + new java.math.BigDecimal("1.00000")), + ("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(1), + Array[Byte](0x0A, 0x0B, 0x0C)) ).foreach { case (query, getter, expectedValue) => var resultSet: Option[ResultSet] = None withExecuteQuery(s"SELECT $query") { rs => assert(rs.next()) - assert(getter(rs) === expectedValue) + expectedValue match { + case arr: Array[Byte] => assert(getter(rs).asInstanceOf[Array[Byte]].sameElements(arr)) + case other => assert(getter(rs) === other) + } assert(!rs.wasNull) resultSet = Some(rs) } @@ -410,4 +416,70 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess assert(!rs.next()) } } + + test("get binary type") { + val testBytes = Array[Byte](0x01, 0x02, 0x03, 0x04, 0x05) + val hexString = testBytes.map(b => "%02X".format(b)).mkString + withExecuteQuery(s"SELECT CAST(X'$hexString' AS BINARY)") { rs => + assert(rs.next()) + val bytes = rs.getBytes(1) + assert(bytes !== null) + assert(bytes.length === testBytes.length) + assert(bytes.sameElements(testBytes)) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnType(1) === Types.BINARY) + assert(metaData.getColumnTypeName(1) === "BINARY") + assert(metaData.getColumnClassName(1) === "[B") + assert(metaData.isSigned(1) === false) + } + } + + test("get binary type with null") { + withExecuteQuery("SELECT cast(null as binary)") { rs => + assert(rs.next()) + assert(rs.getBytes(1) === null) + assert(rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnType(1) === Types.BINARY) + assert(metaData.getColumnTypeName(1) === "BINARY") + assert(metaData.getColumnClassName(1) === "[B") + } + } + + test("get binary type by column label") { + val testBytes = Array[Byte](0x0A, 0x0B, 0x0C) + val hexString = testBytes.map(b => "%02X".format(b)).mkString + withExecuteQuery(s"SELECT CAST(X'$hexString' AS BINARY) as test_binary") { rs => + assert(rs.next()) + val bytes = rs.getBytes("test_binary") + assert(bytes !== null) + assert(bytes.length === testBytes.length) + assert(bytes.sameElements(testBytes)) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "test_binary") + assert(metaData.getColumnLabel(1) === "test_binary") + } + } + + test("get empty binary") { + withExecuteQuery("SELECT CAST(X'' AS BINARY)") { rs => + assert(rs.next()) + val bytes = rs.getBytes(1) + assert(bytes !== null) + assert(bytes.length === 0) + assert(!rs.wasNull) + assert(!rs.next()) + } + } } From e615852fbd7a80559130ef6fa21717d6480e8bc0 Mon Sep 17 00:00:00 2001 From: Uros Stankovic Date: Wed, 12 Nov 2025 11:46:03 -0800 Subject: [PATCH 119/400] [SPARK-54240] Translate get array item catalyst expression to connector expression ### What changes were proposed in this pull request? - Support conversion of catalyst GetArrayItem expression to connector expression to allow data sources to implement pushdown of this expression ### Why are the changes needed? - To allow data sources (built-in and third-party) to implement pushdown of get array item ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No testing needed, since we did not implement pushdowns yet ### Was this patch authored or co-authored using generative AI tooling? No Closes #52940 from urosstan-db/SPARK-54240-support-get-array-item-pushdown. Lead-authored-by: Uros Stankovic Co-authored-by: Wenchen Fan Co-authored-by: Uros Stankovic <155642965+urosstan-db@users.noreply.github.com> Signed-off-by: Wenchen Fan (cherry picked from commit 05b054315b8448d0e650892c3808b885a06d5344) Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 6 ++ .../connector/expressions/GetArrayItem.java | 55 +++++++++++++++++++ .../util/V2ExpressionSQLBuilder.java | 10 ++++ .../catalyst/util/V2ExpressionBuilder.scala | 9 ++- .../connector/ToStringSQLBuilder.scala | 5 ++ 5 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GetArrayItem.java diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 423189f782580..ced4d70ab8ddc 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1688,6 +1688,12 @@ ], "sqlState" : "42846" }, + "EXPRESSION_TRANSLATION_TO_V2_IS_NOT_SUPPORTED" : { + "message" : [ + "Expression cannot be translated to v2 expression." + ], + "sqlState" : "0A000" + }, "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : { "message" : [ "Column expression cannot be sorted because its type is not orderable." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GetArrayItem.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GetArrayItem.java new file mode 100644 index 0000000000000..5d7e05f598adc --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GetArrayItem.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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.internal.connector.ExpressionWithToString; + +/** + * Get array item expression. + * + * @since 4.1.0 + */ + +@Evolving +public class GetArrayItem extends ExpressionWithToString { + + private final Expression childArray; + private final Expression ordinal; + private final boolean failOnError; + + /** + * Creates GetArrayItem expression. + * @param childArray Array that is source to get element from. Child of this expression. + * @param ordinal Ordinal of element. Zero-based indexing. + * @param failOnError Whether expression should throw exception for index out of bound or to + * return null. + */ + public GetArrayItem(Expression childArray, Expression ordinal, boolean failOnError) { + this.childArray = childArray; + this.ordinal = ordinal; + this.failOnError = failOnError; + } + + public Expression childArray() { return this.childArray; } + public Expression ordinal() { return this.ordinal; } + public boolean failOnError() { return this.failOnError; } + + @Override + public Expression[] children() { return new Expression[]{ childArray, ordinal }; } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java index e3b8754691693..50921f3de0b40 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java @@ -29,6 +29,7 @@ import org.apache.spark.sql.connector.expressions.Extract; import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.expressions.GeneralScalarExpression; +import org.apache.spark.sql.connector.expressions.GetArrayItem; import org.apache.spark.sql.connector.expressions.Literal; import org.apache.spark.sql.connector.expressions.NullOrdering; import org.apache.spark.sql.connector.expressions.SortDirection; @@ -84,6 +85,8 @@ public String build(Expression expr) { } else if (expr instanceof SortOrder sortOrder) { return visitSortOrder( build(sortOrder.expression()), sortOrder.direction(), sortOrder.nullOrdering()); + } else if (expr instanceof GetArrayItem getArrayItem) { + return visitGetArrayItem(getArrayItem); } else if (expr instanceof GeneralScalarExpression e) { String name = e.name(); return switch (name) { @@ -348,6 +351,13 @@ protected String visitTrim(String direction, String[] inputs) { } } + protected String visitGetArrayItem(GetArrayItem getArrayItem) { + throw new SparkUnsupportedOperationException( + "EXPRESSION_TRANSLATION_TO_V2_IS_NOT_SUPPORTED", + Map.of("expr", getArrayItem.toString()) + ); + } + protected String visitExtract(Extract extract) { return visitExtract(extract.field(), build(extract.source())); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala index 4e391208d984b..72b466f5a0f9a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} import org.apache.spark.sql.catalyst.optimizer.ConstantFolding import org.apache.spark.sql.connector.catalog.functions.ScalarFunction -import org.apache.spark.sql.connector.expressions.{Cast => V2Cast, Expression => V2Expression, Extract => V2Extract, FieldReference, GeneralScalarExpression, LiteralValue, NullOrdering, SortDirection, SortValue, UserDefinedScalarFunc} +import org.apache.spark.sql.connector.expressions.{Cast => V2Cast, Expression => V2Expression, Extract => V2Extract, FieldReference, GeneralScalarExpression, GetArrayItem => V2GetArrayItem, LiteralValue, NullOrdering, SortDirection, SortValue, UserDefinedScalarFunc} import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Avg, Count, CountStar, GeneralAggregateFunc, Max, Min, Sum, UserDefinedAggregateFunc} import org.apache.spark.sql.connector.expressions.filter.{AlwaysFalse, AlwaysTrue, And => V2And, Not => V2Not, Or => V2Or, Predicate => V2Predicate} import org.apache.spark.sql.internal.SQLConf @@ -326,6 +326,13 @@ class V2ExpressionBuilder(e: Expression, isPredicate: Boolean = false) extends L case _: Sha2 => generateExpressionWithName("SHA2", expr, isPredicate) case _: StringLPad => generateExpressionWithName("LPAD", expr, isPredicate) case _: StringRPad => generateExpressionWithName("RPAD", expr, isPredicate) + case GetArrayItem(child, ordinal, failOnError) => + (generateExpression(child), generateExpression(ordinal)) match { + case (Some(v2ArrayChild), Some(v2Ordinal)) => + Some(new V2GetArrayItem(v2ArrayChild, v2Ordinal, failOnError)) + case _ => + None + } // TODO supports other expressions case ApplyFunctionExpression(function, children) => val childrenExpressions = children.flatMap(generateExpression(_)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ToStringSQLBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ToStringSQLBuilder.scala index 118c1af977454..5c54f28976458 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ToStringSQLBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ToStringSQLBuilder.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.internal.connector +import org.apache.spark.sql.connector.expressions.GetArrayItem import org.apache.spark.sql.connector.util.V2ExpressionSQLBuilder /** @@ -35,4 +36,8 @@ class ToStringSQLBuilder extends V2ExpressionSQLBuilder with Serializable { val distinct = if (isDistinct) "DISTINCT " else "" s"""$funcName($distinct${inputs.mkString(", ")})""" } + + override protected def visitGetArrayItem(getArrayItem: GetArrayItem): String = { + s"${getArrayItem.childArray.toString}[${getArrayItem.ordinal.toString}]" + } } From 222c750d1cdcf32535bba95a507dd833ef1e2e0e Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Wed, 12 Nov 2025 14:20:27 -0800 Subject: [PATCH 120/400] [SPARK-54323][PYTHON] Change the way to access logs to TVF instead of system view ### What changes were proposed in this pull request? Changes the way to access logs to TVF instead of system view. ```sql SELECT * FROM python_worker_logs() ``` ```py spark.tvf.python_worker_logs() ``` Also blocks the TVF when the python worker logging is disabled. ```py >>> spark.conf.get('spark.sql.pyspark.worker.logging.enabled') 'false' >>> spark.tvf.python_worker_logs().show() Traceback (most recent call last): ... pyspark.errors.exceptions.captured.AnalysisException: [FEATURE_NOT_ENABLED] The feature Python Worker Logging is not enabled. Consider setting the config spark.sql.pyspark.worker.logging.enabled to true to enable this capability. SQLSTATE: 56038 >>> spark.conf.set('spark.sql.pyspark.worker.logging.enabled', True) >>> spark.tvf.python_worker_logs().show() +---+-----+---+-------+---------+------+ | ts|level|msg|context|exception|logger| +---+-----+---+-------+---------+------+ +---+-----+---+-------+---------+------+ ``` ### Why are the changes needed? There may be namespace conflicts with the other system tables/views, etc.. For example, the variables in SQL has the same namespace `system.session.varname`, which may potentially cause an issue. ### Does this PR introduce _any_ user-facing change? Yes, the way to access python worker logs will be changed. ### How was this patch tested? Modified the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53021 from ueshin/issues/SPARK-54323/tvf. Authored-by: Takuya Ueshin Signed-off-by: Dongjoon Hyun (cherry picked from commit fb8dbd7071b13549d672cc204cec7ab12e7dc520) Signed-off-by: Dongjoon Hyun --- project/MimaExcludes.scala | 5 +- .../docs/source/user_guide/bugbusting.ipynb | 12 +- python/pyspark/sql/connect/tvf.py | 5 + .../tests/arrow/test_arrow_cogrouped_map.py | 28 +- .../sql/tests/arrow/test_arrow_grouped_map.py | 54 ++-- .../pyspark/sql/tests/arrow/test_arrow_map.py | 10 +- .../tests/arrow/test_arrow_udf_grouped_agg.py | 26 +- .../sql/tests/arrow/test_arrow_udf_scalar.py | 52 +-- .../sql/tests/arrow/test_arrow_udf_window.py | 28 +- .../sql/tests/arrow/test_arrow_udtf.py | 26 +- .../tests/pandas/test_pandas_cogrouped_map.py | 28 +- .../tests/pandas/test_pandas_grouped_map.py | 26 +- .../sql/tests/pandas/test_pandas_map.py | 28 +- .../pandas/test_pandas_udf_grouped_agg.py | 26 +- .../tests/pandas/test_pandas_udf_scalar.py | 52 +-- .../tests/pandas/test_pandas_udf_window.py | 28 +- .../sql/tests/test_python_datasource.py | 298 +++++++++--------- python/pyspark/sql/tests/test_udf.py | 142 +++++---- python/pyspark/sql/tests/test_udtf.py | 106 +++---- python/pyspark/sql/tvf.py | 39 +++ .../spark/sql/TableValuedFunction.scala | 8 + .../catalyst/analysis/FunctionRegistry.scala | 5 +- .../analysis/RelationResolution.scala | 20 +- .../logical/pythonLogicalOperators.scala | 64 ++-- .../sql/errors/QueryCompilationErrors.scala | 11 + .../sql/connect/TableValuedFunction.scala | 4 + .../sql/classic/TableValuedFunction.scala | 4 + .../python/PythonWorkerLogsSuite.scala | 30 +- 28 files changed, 642 insertions(+), 523 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1614ec212c2e8..2e4598810451c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -48,7 +48,10 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.Dataset.repartitionById"), // [SPARK-54001][CONNECT] Replace block copying with ref-counting in ArtifactManager cloning - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.artifact.ArtifactManager.cachedBlockIdList") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.artifact.ArtifactManager.cachedBlockIdList"), + + // [SPARK-54323][PYTHON] Change the way to access logs to TVF instead of system view + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.TableValuedFunction.python_worker_logs") ) // Default exclude rules diff --git a/python/docs/source/user_guide/bugbusting.ipynb b/python/docs/source/user_guide/bugbusting.ipynb index 75150596600ef..407bcbf18e9b3 100644 --- a/python/docs/source/user_guide/bugbusting.ipynb +++ b/python/docs/source/user_guide/bugbusting.ipynb @@ -945,7 +945,7 @@ "metadata": {}, "outputs": [], "source": [ - "logs = spark.table(\"system.session.python_worker_logs\")" + "logs = spark.tvf.python_worker_logs()" ] }, { @@ -1021,7 +1021,7 @@ "df.select(my_udf(\"text\")).show()\n", "\n", "# Query the logs\n", - "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs = spark.tvf.python_worker_logs()\n", "logs.select(\"level\", \"msg\", \"logger\", \"context\").show(truncate=False)" ] }, @@ -1076,7 +1076,7 @@ "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")\n", "spark.range(1).select(contextual_udf(lit(\"test\"))).show()\n", "\n", - "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs = spark.tvf.python_worker_logs()\n", "logs.filter(\"logger = 'contextual'\").select(\"msg\", \"context\").show(truncate=False)" ] }, @@ -1135,7 +1135,7 @@ "spark.conf.set(\"spark.sql.pyspark.worker.logging.enabled\", \"true\")\n", "spark.createDataFrame([(0,), (5,)], [\"value\"]).select(failing_udf(\"value\")).show()\n", "\n", - "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs = spark.tvf.python_worker_logs()\n", "logs.filter(\"logger = 'error_handler'\").select(\"msg\", \"exception\").show(truncate=False)" ] }, @@ -1193,7 +1193,7 @@ "df = spark.createDataFrame([(\"hello world\",)], [\"text\"])\n", "df.lateralJoin(WordSplitter(col(\"text\").outer())).show()\n", "\n", - "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs = spark.tvf.python_worker_logs()\n", "logs.filter(\"logger = 'udtf_logger'\").select(\"msg\", \"context\").show(truncate=False)" ] }, @@ -1231,7 +1231,7 @@ } ], "source": [ - "logs = spark.table(\"system.session.python_worker_logs\")\n", + "logs = spark.tvf.python_worker_logs()\n", "\n", "# Count logs by level\n", "logs.groupBy(\"level\").count().show()\n", diff --git a/python/pyspark/sql/connect/tvf.py b/python/pyspark/sql/connect/tvf.py index 59a4e4fbe344e..ac41b0b21725f 100644 --- a/python/pyspark/sql/connect/tvf.py +++ b/python/pyspark/sql/connect/tvf.py @@ -109,6 +109,11 @@ def variant_explode_outer(self, input: "Column") -> "DataFrame": variant_explode_outer.__doc__ = PySparkTableValuedFunction.variant_explode_outer.__doc__ + def python_worker_logs(self) -> "DataFrame": + return self._fn("python_worker_logs") + + python_worker_logs.__doc__ = PySparkTableValuedFunction.python_worker_logs.__doc__ + def _fn(self, name: str, *args: "Column") -> "DataFrame": from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.connect.plan import UnresolvedTableValuedFunction diff --git a/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py b/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py index 6c1c378dcea8e..abfd1af7a741e 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py @@ -396,20 +396,20 @@ def func_with_logging(left, right): + [Row(id=2, v1=20, v2=200)], ) - logs = self.spark.table("system.session.python_worker_logs") - - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"arrow cogrouped map: {dict(v1=v1, v2=v2)}", - context={"func_name": func_with_logging.__name__}, - logger="test_arrow_cogrouped_map", - ) - for v1, v2 in [([10, 30], [100, 300]), ([20], [200])] - ], - ) + logs = self.spark.tvf.python_worker_logs() + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"arrow cogrouped map: {dict(v1=v1, v2=v2)}", + context={"func_name": func_with_logging.__name__}, + logger="test_arrow_cogrouped_map", + ) + for v1, v2 in [([10, 30], [100, 300]), ([20], [200])] + ], + ) class CogroupedMapInArrowTests(CogroupedMapInArrowTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py b/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py index 829c38385bd0e..56a9ed322c154 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py @@ -416,20 +416,20 @@ def func_with_logging(group): df, ) - logs = self.spark.table("system.session.python_worker_logs") - - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"arrow grouped map: {dict(id=lst, value=[v*10 for v in lst])}", - context={"func_name": func_with_logging.__name__}, - logger="test_arrow_grouped_map", - ) - for lst in [[0, 2, 4, 6, 8], [1, 3, 5, 7]] - ], - ) + logs = self.spark.tvf.python_worker_logs() + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"arrow grouped map: {dict(id=lst, value=[v*10 for v in lst])}", + context={"func_name": func_with_logging.__name__}, + logger="test_arrow_grouped_map", + ) + for lst in [[0, 2, 4, 6, 8], [1, 3, 5, 7]] + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_apply_in_arrow_iter_with_logging(self): @@ -456,20 +456,20 @@ def func_with_logging(group: Iterator[pa.RecordBatch]) -> Iterator[pa.RecordBatc df, ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"arrow grouped map: {dict(id=lst, value=[v*10 for v in lst])}", - context={"func_name": func_with_logging.__name__}, - logger="test_arrow_grouped_map", - ) - for lst in [[0, 2, 4], [6, 8], [1, 3, 5], [7]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"arrow grouped map: {dict(id=lst, value=[v*10 for v in lst])}", + context={"func_name": func_with_logging.__name__}, + logger="test_arrow_grouped_map", + ) + for lst in [[0, 2, 4], [6, 8], [1, 3, 5], [7]] + ], + ) class ApplyInArrowTests(ApplyInArrowTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/arrow/test_arrow_map.py b/python/pyspark/sql/tests/arrow/test_arrow_map.py index 4a56a32fbcddb..31851e8a8adb9 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_map.py @@ -247,12 +247,12 @@ def func_with_logging(iterator): [Row(id=i) for i in range(9)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - self._expected_logs_for_test_map_in_arrow_with_logging(func_with_logging.__name__), - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + self._expected_logs_for_test_map_in_arrow_with_logging(func_with_logging.__name__), + ) def _expected_logs_for_test_map_in_arrow_with_logging(self, func_name): return [ diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py index f719b4fb16bd2..3d8588ffb7af3 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py @@ -1044,20 +1044,20 @@ def my_grouped_agg_arrow_udf(x): [Row(id=1, result=3.0), Row(id=2, result=18.0)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"grouped agg arrow udf: {n}", - context={"func_name": my_grouped_agg_arrow_udf.__name__}, - logger="test_grouped_agg_arrow", - ) - for n in [2, 3] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"grouped agg arrow udf: {n}", + context={"func_name": my_grouped_agg_arrow_udf.__name__}, + logger="test_grouped_agg_arrow", + ) + for n in [2, 3] + ], + ) class GroupedAggArrowUDFTests(GroupedAggArrowUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py index 05f33a4ae42f7..3bd00d2cc921a 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py @@ -1201,20 +1201,20 @@ def my_scalar_arrow_udf(x): [Row(result=f"scalar_arrow_{i}") for i in range(3)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"scalar arrow udf: {lst}", - context={"func_name": my_scalar_arrow_udf.__name__}, - logger="test_scalar_arrow", - ) - for lst in [[0], [1, 2]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"scalar arrow udf: {lst}", + context={"func_name": my_scalar_arrow_udf.__name__}, + logger="test_scalar_arrow", + ) + for lst in [[0], [1, 2]] + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_scalar_iter_arrow_udf_with_logging(self): @@ -1241,20 +1241,20 @@ def my_scalar_iter_arrow_udf(it): [Row(result=f"scalar_iter_arrow_{i}") for i in range(9)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"scalar iter arrow udf: {lst}", - context={"func_name": my_scalar_iter_arrow_udf.__name__}, - logger="test_scalar_iter_arrow", - ) - for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"scalar iter arrow udf: {lst}", + context={"func_name": my_scalar_iter_arrow_udf.__name__}, + logger="test_scalar_iter_arrow", + ) + for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] + ], + ) class ScalarArrowUDFTests(ScalarArrowUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py index 240e34487b006..b0adfbe131864 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py @@ -834,20 +834,20 @@ def my_window_arrow_udf(x): ], ) - logs = self.spark.table("system.session.python_worker_logs") - - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"window arrow udf: {lst}", - context={"func_name": my_window_arrow_udf.__name__}, - logger="test_window_arrow", - ) - for lst in [[1.0], [1.0, 2.0], [3.0], [3.0, 5.0], [3.0, 5.0, 10.0]] - ], - ) + logs = self.spark.tvf.python_worker_logs() + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"window arrow udf: {lst}", + context={"func_name": my_window_arrow_udf.__name__}, + logger="test_window_arrow", + ) + for lst in [[1.0], [1.0, 2.0], [3.0], [3.0, 5.0], [3.0, 5.0, 10.0]] + ], + ) class WindowArrowUDFTests(WindowArrowUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py index 24c5ec0acf91e..cc0edda378abf 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py @@ -1721,20 +1721,20 @@ def eval(self, table_data: "pa.RecordBatch") -> Iterator["pa.Table"]: [Row(id=i, doubled=i * 2) for i in range(9)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"arrow udtf: {dict(id=lst)}", - context={"class_name": "TestArrowUDTFWithLogging", "func_name": "eval"}, - logger="test_arrow_udtf", - ) - for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"arrow udtf: {dict(id=lst)}", + context={"class_name": "TestArrowUDTFWithLogging", "func_name": "eval"}, + logger="test_arrow_udtf", + ) + for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] + ], + ) class ArrowUDTFTests(ArrowUDTFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py index 8d48bbb794ada..0d91da0354979 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py @@ -742,20 +742,20 @@ def func_with_logging(left_pdf, right_pdf): + [Row(id=2, v1=20, v2=200)], ) - logs = self.spark.table("system.session.python_worker_logs") - - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"pandas cogrouped map: {dict(v1=v1, v2=v2)}", - context={"func_name": func_with_logging.__name__}, - logger="test_pandas_cogrouped_map", - ) - for v1, v2 in [([10, 30], [100, 300]), ([20], [200])] - ], - ) + logs = self.spark.tvf.python_worker_logs() + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"pandas cogrouped map: {dict(v1=v1, v2=v2)}", + context={"func_name": func_with_logging.__name__}, + logger="test_pandas_cogrouped_map", + ) + for v1, v2 in [([10, 30], [100, 300]), ([20], [200])] + ], + ) class CogroupedApplyInPandasTests(CogroupedApplyInPandasTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index 79895bb1a01b3..8bf0e7235efcf 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -1000,20 +1000,20 @@ def func_with_logging(pdf): df, ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"pandas grouped map: {dict(id=lst, value=[v*10 for v in lst])}", - context={"func_name": func_with_logging.__name__}, - logger="test_pandas_grouped_map", - ) - for lst in [[0, 2, 4, 6, 8], [1, 3, 5, 7]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"pandas grouped map: {dict(id=lst, value=[v*10 for v in lst])}", + context={"func_name": func_with_logging.__name__}, + logger="test_pandas_grouped_map", + ) + for lst in [[0, 2, 4, 6, 8], [1, 3, 5, 7]] + ], + ) def test_apply_in_pandas_iterator_basic(self): df = self.spark.createDataFrame( diff --git a/python/pyspark/sql/tests/pandas/test_pandas_map.py b/python/pyspark/sql/tests/pandas/test_pandas_map.py index 5e0e33a05b22b..946d56f2fe637 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_map.py @@ -510,20 +510,20 @@ def func_with_logging(iterator): [Row(id=i) for i in range(9)], ) - logs = self.spark.table("system.session.python_worker_logs") - - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"pandas map: {lst}", - context={"func_name": func_with_logging.__name__}, - logger="test_pandas_map", - ) - for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] - ], - ) + logs = self.spark.tvf.python_worker_logs() + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"pandas map: {lst}", + context={"func_name": func_with_logging.__name__}, + logger="test_pandas_map", + ) + for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] + ], + ) class MapInPandasTests(ReusedSQLTestCase, MapInPandasTestsMixin): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py index 6915e8aee948d..4b66dee5b7af5 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py @@ -846,20 +846,20 @@ def my_grouped_agg_pandas_udf(x): [Row(id=1, result=3.0), Row(id=2, result=18.0)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"grouped agg pandas udf: {n}", - context={"func_name": my_grouped_agg_pandas_udf.__name__}, - logger="test_grouped_agg_pandas", - ) - for n in [2, 3] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"grouped agg pandas udf: {n}", + context={"func_name": my_grouped_agg_pandas_udf.__name__}, + logger="test_grouped_agg_pandas", + ) + for n in [2, 3] + ], + ) def test_grouped_agg_pandas_udf_with_compression_codec(self): # Test grouped agg Pandas UDF with different compression codec settings diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py index 563e0b789a962..b936a9240e529 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py @@ -1935,20 +1935,20 @@ def my_scalar_pandas_udf(x): [Row(result=f"scalar_pandas_{i}") for i in range(3)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"scalar pandas udf: {lst}", - context={"func_name": my_scalar_pandas_udf.__name__}, - logger="test_scalar_pandas", - ) - for lst in [[0], [1, 2]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"scalar pandas udf: {lst}", + context={"func_name": my_scalar_pandas_udf.__name__}, + logger="test_scalar_pandas", + ) + for lst in [[0], [1, 2]] + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_scalar_iter_pandas_udf_with_logging(self): @@ -1973,20 +1973,20 @@ def my_scalar_iter_pandas_udf(it): [Row(result=f"scalar_iter_pandas_{i}") for i in range(9)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"scalar iter pandas udf: {lst}", - context={"func_name": my_scalar_iter_pandas_udf.__name__}, - logger="test_scalar_iter_pandas", - ) - for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"scalar iter pandas udf: {lst}", + context={"func_name": my_scalar_iter_pandas_udf.__name__}, + logger="test_scalar_iter_pandas", + ) + for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]] + ], + ) def test_scalar_pandas_udf_with_compression_codec(self): # Test scalar Pandas UDF with different compression codec settings diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py index 6fa7e9063836b..6e1cbdaf73cff 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py @@ -664,20 +664,20 @@ def my_window_pandas_udf(x): ], ) - logs = self.spark.table("system.session.python_worker_logs") - - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"window pandas udf: {lst}", - context={"func_name": my_window_pandas_udf.__name__}, - logger="test_window_pandas", - ) - for lst in [[1.0], [1.0, 2.0], [3.0], [3.0, 5.0], [3.0, 5.0, 10.0]] - ], - ) + logs = self.spark.tvf.python_worker_logs() + + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"window pandas udf: {lst}", + context={"func_name": my_window_pandas_udf.__name__}, + logger="test_window_pandas", + ) + for lst in [[1.0], [1.0, 2.0], [3.0], [3.0, 5.0], [3.0, 5.0, 10.0]] + ], + ) class WindowPandasUDFTests(WindowPandasUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/test_python_datasource.py b/python/pyspark/sql/tests/test_python_datasource.py index 532f68f2c8f26..e5171876656c7 100644 --- a/python/pyspark/sql/tests/test_python_datasource.py +++ b/python/pyspark/sql/tests/test_python_datasource.py @@ -965,49 +965,49 @@ def reader(self, schema) -> "DataSourceReader": ], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=msg, - context=context, - logger="test_data_source_reader", - ) - for msg, context in [ - ( - "TestJsonDataSource.__init__: ['path']", - {"class_name": "TestJsonDataSource", "func_name": "__init__"}, - ), - ( - "TestJsonDataSource.name", - {"class_name": "TestJsonDataSource", "func_name": "name"}, - ), - ( - "TestJsonDataSource.schema", - {"class_name": "TestJsonDataSource", "func_name": "schema"}, - ), - ( - "TestJsonDataSource.reader: ['name', 'age']", - {"class_name": "TestJsonDataSource", "func_name": "reader"}, - ), - ( - "TestJsonReader.__init__: ['path']", - {"class_name": "TestJsonDataSource", "func_name": "reader"}, - ), - ( - "TestJsonReader.partitions", - {"class_name": "TestJsonReader", "func_name": "partitions"}, - ), - ( - "TestJsonReader.read: None", - {"class_name": "TestJsonReader", "func_name": "read"}, - ), - ] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=msg, + context=context, + logger="test_data_source_reader", + ) + for msg, context in [ + ( + "TestJsonDataSource.__init__: ['path']", + {"class_name": "TestJsonDataSource", "func_name": "__init__"}, + ), + ( + "TestJsonDataSource.name", + {"class_name": "TestJsonDataSource", "func_name": "name"}, + ), + ( + "TestJsonDataSource.schema", + {"class_name": "TestJsonDataSource", "func_name": "schema"}, + ), + ( + "TestJsonDataSource.reader: ['name', 'age']", + {"class_name": "TestJsonDataSource", "func_name": "reader"}, + ), + ( + "TestJsonReader.__init__: ['path']", + {"class_name": "TestJsonDataSource", "func_name": "reader"}, + ), + ( + "TestJsonReader.partitions", + {"class_name": "TestJsonReader", "func_name": "partitions"}, + ), + ( + "TestJsonReader.read: None", + {"class_name": "TestJsonReader", "func_name": "read"}, + ), + ] + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_data_source_reader_pushdown_with_logging(self): @@ -1072,53 +1072,53 @@ def reader(self, schema) -> "DataSourceReader": ], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=msg, - context=context, - logger="test_data_source_reader_pushdown", - ) - for msg, context in [ - ( - "TestJsonDataSource.__init__: ['path']", - {"class_name": "TestJsonDataSource", "func_name": "__init__"}, - ), - ( - "TestJsonDataSource.name", - {"class_name": "TestJsonDataSource", "func_name": "name"}, - ), - ( - "TestJsonDataSource.schema", - {"class_name": "TestJsonDataSource", "func_name": "schema"}, - ), - ( - "TestJsonDataSource.reader: ['name', 'age']", - {"class_name": "TestJsonDataSource", "func_name": "reader"}, - ), - ( - "TestJsonReader.pushFilters: [IsNotNull(attribute=('age',))]", - {"class_name": "TestJsonReader", "func_name": "pushFilters"}, - ), - ( - "TestJsonReader.__init__: ['path']", - {"class_name": "TestJsonDataSource", "func_name": "reader"}, - ), - ( - "TestJsonReader.partitions", - {"class_name": "TestJsonReader", "func_name": "partitions"}, - ), - ( - "TestJsonReader.read: None", - {"class_name": "TestJsonReader", "func_name": "read"}, - ), - ] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=msg, + context=context, + logger="test_data_source_reader_pushdown", + ) + for msg, context in [ + ( + "TestJsonDataSource.__init__: ['path']", + {"class_name": "TestJsonDataSource", "func_name": "__init__"}, + ), + ( + "TestJsonDataSource.name", + {"class_name": "TestJsonDataSource", "func_name": "name"}, + ), + ( + "TestJsonDataSource.schema", + {"class_name": "TestJsonDataSource", "func_name": "schema"}, + ), + ( + "TestJsonDataSource.reader: ['name', 'age']", + {"class_name": "TestJsonDataSource", "func_name": "reader"}, + ), + ( + "TestJsonReader.pushFilters: [IsNotNull(attribute=('age',))]", + {"class_name": "TestJsonReader", "func_name": "pushFilters"}, + ), + ( + "TestJsonReader.__init__: ['path']", + {"class_name": "TestJsonDataSource", "func_name": "reader"}, + ), + ( + "TestJsonReader.partitions", + {"class_name": "TestJsonReader", "func_name": "partitions"}, + ), + ( + "TestJsonReader.read: None", + {"class_name": "TestJsonReader", "func_name": "read"}, + ), + ] + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_data_source_writer_with_logging(self): @@ -1197,69 +1197,69 @@ def writer(self, schema, overwrite): with self.assertRaises(Exception, msg="abort test"): df.write.format("my-json").mode("append").option("abort", "true").save(d) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=msg, - context=context, - logger="test_datasource_writer", + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=msg, + context=context, + logger="test_datasource_writer", + ) + for msg, context in [ + ( + "TestJsonDataSource.name", + {"class_name": "TestJsonDataSource", "func_name": "name"}, + ), + ( + "TestJsonDataSource.writer: (['name', 'age'], {True})", + {"class_name": "TestJsonDataSource", "func_name": "writer"}, + ), + ( + "TestJsonWriter.__init__: ['path']", + {"class_name": "TestJsonDataSource", "func_name": "writer"}, + ), + ( + "TestJsonWriter.write: 1, [{'name': 'Diana', 'age': 28}]", + {"class_name": "TestJsonWriter", "func_name": "write"}, + ), + ( + "TestJsonWriter.write: 1, [{'name': 'Charlie', 'age': 35}]", + {"class_name": "TestJsonWriter", "func_name": "write"}, + ), + ( + "TestJsonWriter.commit: 2", + {"class_name": "TestJsonWriter", "func_name": "commit"}, + ), + ( + "TestJsonDataSource.name", + {"class_name": "TestJsonDataSource", "func_name": "name"}, + ), + ( + "TestJsonDataSource.writer: (['name', 'age'], {False})", + {"class_name": "TestJsonDataSource", "func_name": "writer"}, + ), + ( + "TestJsonWriter.__init__: ['abort', 'path']", + {"class_name": "TestJsonDataSource", "func_name": "writer"}, + ), + ( + "TestJsonWriter.write: abort test", + {"class_name": "TestJsonWriter", "func_name": "write"}, + ), + ( + "TestJsonWriter.write: abort test", + {"class_name": "TestJsonWriter", "func_name": "write"}, + ), + ( + "TestJsonWriter.abort", + {"class_name": "TestJsonWriter", "func_name": "abort"}, + ), + ] + ], ) - for msg, context in [ - ( - "TestJsonDataSource.name", - {"class_name": "TestJsonDataSource", "func_name": "name"}, - ), - ( - "TestJsonDataSource.writer: (['name', 'age'], {True})", - {"class_name": "TestJsonDataSource", "func_name": "writer"}, - ), - ( - "TestJsonWriter.__init__: ['path']", - {"class_name": "TestJsonDataSource", "func_name": "writer"}, - ), - ( - "TestJsonWriter.write: 1, [{'name': 'Diana', 'age': 28}]", - {"class_name": "TestJsonWriter", "func_name": "write"}, - ), - ( - "TestJsonWriter.write: 1, [{'name': 'Charlie', 'age': 35}]", - {"class_name": "TestJsonWriter", "func_name": "write"}, - ), - ( - "TestJsonWriter.commit: 2", - {"class_name": "TestJsonWriter", "func_name": "commit"}, - ), - ( - "TestJsonDataSource.name", - {"class_name": "TestJsonDataSource", "func_name": "name"}, - ), - ( - "TestJsonDataSource.writer: (['name', 'age'], {False})", - {"class_name": "TestJsonDataSource", "func_name": "writer"}, - ), - ( - "TestJsonWriter.__init__: ['abort', 'path']", - {"class_name": "TestJsonDataSource", "func_name": "writer"}, - ), - ( - "TestJsonWriter.write: abort test", - {"class_name": "TestJsonWriter", "func_name": "write"}, - ), - ( - "TestJsonWriter.write: abort test", - {"class_name": "TestJsonWriter", "func_name": "write"}, - ), - ( - "TestJsonWriter.abort", - {"class_name": "TestJsonWriter", "func_name": "abort"}, - ), - ] - ], - ) class PythonDataSourceTests(BasePythonDataSourceTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 0ceb745c58607..9c5fa2ad1bba8 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1573,50 +1573,66 @@ def my_udf(): logger.exception("exception") return "x" + # The TVF is not available when the feature is disabled. + with self.assertRaises(AnalysisException) as pe: + self.spark.tvf.python_worker_logs().count() + + self.check_error( + exception=pe.exception, + errorClass="FEATURE_NOT_ENABLED", + messageParameters={ + "featureName": "Python Worker Logging", + "configKey": "spark.sql.pyspark.worker.logging.enabled", + "configValue": "true", + }, + ) + # Logging is disabled by default assertDataFrameEqual( self.spark.range(1).select(my_udf().alias("result")), [Row(result="x")] ) - self.assertEqual(self.spark.table("system.session.python_worker_logs").count(), 0) with self.sql_conf({"spark.sql.pyspark.worker.logging.enabled": "true"}): + # The logs were not collected when the feature was disabled. + self.assertEqual(self.spark.tvf.python_worker_logs().count(), 0) + assertDataFrameEqual( self.spark.range(1).select(my_udf().alias("result")), [Row(result="x")] ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="INFO", - msg="print to stdout ❤", - context={"func_name": my_udf.__name__}, - logger="stdout", - ), - Row( - level="ERROR", - msg="print to stderr 😀", - context={"func_name": my_udf.__name__}, - logger="stderr", - ), - Row( - level="WARNING", - msg="custom context", - context={"func_name": my_udf.__name__, "abc": "123"}, - logger="test", - ), - Row( - level="ERROR", - msg="exception", - context={"func_name": my_udf.__name__}, - logger="test", - ), - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="INFO", + msg="print to stdout ❤", + context={"func_name": my_udf.__name__}, + logger="stdout", + ), + Row( + level="ERROR", + msg="print to stderr 😀", + context={"func_name": my_udf.__name__}, + logger="stderr", + ), + Row( + level="WARNING", + msg="custom context", + context={"func_name": my_udf.__name__, "abc": "123"}, + logger="test", + ), + Row( + level="ERROR", + msg="exception", + context={"func_name": my_udf.__name__}, + logger="test", + ), + ], + ) - self.assertEqual(logs.where("exception is not null").select("exception").count(), 1) + self.assertEqual(logs.where("exception is not null").select("exception").count(), 1) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_multiple_udfs_with_logging(self): @@ -1638,25 +1654,25 @@ def my_udf2(): [Row(result="x", result2="y")], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg="test1", - context={"func_name": my_udf1.__name__}, - logger="test1", - ), - Row( - level="WARNING", - msg="test2", - context={"func_name": my_udf2.__name__}, - logger="test2", - ), - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg="test1", + context={"func_name": my_udf1.__name__}, + logger="test1", + ), + Row( + level="WARNING", + msg="test2", + context={"func_name": my_udf2.__name__}, + logger="test2", + ), + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_udf_with_pyspark_logger(self): @@ -1672,20 +1688,20 @@ def my_udf(x): [Row(result=str(i)) for i in range(2)], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg="PySparkLogger test", - context={"func_name": my_udf.__name__, "x": str(i)}, - logger="PySparkLogger", - ) - for i in range(2) - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg="PySparkLogger test", + context={"func_name": my_udf.__name__, "x": str(i)}, + logger="PySparkLogger", + ) + for i in range(2) + ], + ) class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 70623a5f62d44..5ded5aa67b4eb 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3079,20 +3079,20 @@ def eval(self, x: int): [Row(x=x, a=x * 2, b=x + 10) for x in [5, 10]], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select("level", "msg", "context", "logger"), - [ - Row( - level="WARNING", - msg=f"udtf with logging: {x}", - context={"class_name": "TestUDTFWithLogging", "func_name": "eval"}, - logger="test_udtf", - ) - for x in [5, 10] - ], - ) + assertDataFrameEqual( + logs.select("level", "msg", "context", "logger"), + [ + Row( + level="WARNING", + msg=f"udtf with logging: {x}", + context={"class_name": "TestUDTFWithLogging", "func_name": "eval"}, + logger="test_udtf", + ) + for x in [5, 10] + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_udtf_analyze_with_logging(self): @@ -3115,26 +3115,26 @@ def eval(self, x: int): [Row(x=x, a=x * 2, b=x + 10) for x in [5, 10]], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select( - "level", - "msg", - col("context.class_name").alias("context_class_name"), - col("context.func_name").alias("context_func_name"), - "logger", - ).distinct(), - [ - Row( - level="WARNING", - msg='udtf analyze: "long"', - context_class_name="TestUDTFWithLogging", - context_func_name="analyze", - logger="test_udtf", - ) - ], - ) + assertDataFrameEqual( + logs.select( + "level", + "msg", + col("context.class_name").alias("context_class_name"), + col("context.func_name").alias("context_func_name"), + "logger", + ).distinct(), + [ + Row( + level="WARNING", + msg='udtf analyze: "long"', + context_class_name="TestUDTFWithLogging", + context_func_name="analyze", + logger="test_udtf", + ) + ], + ) @unittest.skipIf(is_remote_only(), "Requires JVM access") def test_udtf_analyze_with_pyspark_logger(self): @@ -3157,28 +3157,28 @@ def eval(self, x: int): [Row(x=x, a=x * 2, b=x + 10) for x in [5, 10]], ) - logs = self.spark.table("system.session.python_worker_logs") + logs = self.spark.tvf.python_worker_logs() - assertDataFrameEqual( - logs.select( - "level", - "msg", - col("context.class_name").alias("context_class_name"), - col("context.func_name").alias("context_func_name"), - col("context.dt").alias("context_dt"), - "logger", - ).distinct(), - [ - Row( - level="WARNING", - msg='udtf analyze: "long"', - context_class_name="TestUDTFWithLogging", - context_func_name="analyze", - context_dt='"long"', - logger="PySparkLogger", - ) - ], - ) + assertDataFrameEqual( + logs.select( + "level", + "msg", + col("context.class_name").alias("context_class_name"), + col("context.func_name").alias("context_func_name"), + col("context.dt").alias("context_dt"), + "logger", + ).distinct(), + [ + Row( + level="WARNING", + msg='udtf analyze: "long"', + context_class_name="TestUDTFWithLogging", + context_func_name="analyze", + context_dt='"long"', + logger="PySparkLogger", + ) + ], + ) class UDTFTests(BaseUDTFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tvf.py b/python/pyspark/sql/tvf.py index b34877b03311f..c21e3751655c9 100644 --- a/python/pyspark/sql/tvf.py +++ b/python/pyspark/sql/tvf.py @@ -677,6 +677,45 @@ def variant_explode_outer(self, input: Column) -> DataFrame: """ return self._fn("variant_explode_outer", input) + def python_worker_logs(self) -> DataFrame: + """ + Returns a DataFrame of logs collected from Python workers. + + .. versionadded:: 4.1.0 + + Returns + ------- + :class:`DataFrame` + + Examples + -------- + >>> import pyspark.sql.functions as sf + >>> import logging + >>> + >>> @sf.udf("string") + ... def my_udf(x): + ... logger = logging.getLogger("my_custom_logger") + ... logger.warning("This is a warning") + ... return str(x) + ... + >>> spark.conf.set("spark.sql.pyspark.worker.logging.enabled", "true") + >>> spark.range(1).select(my_udf("id")).show() + +----------+ + |my_udf(id)| + +----------+ + | 0| + +----------+ + >>> spark.tvf.python_worker_logs().select( + ... "level", "msg", "context", "logger" + ... ).show(truncate=False) # doctest: +SKIP + +-------+-----------------+---------------------+----------------+ + |level |msg |context |logger | + +-------+-----------------+---------------------+----------------+ + |WARNING|This is a warning|{func_name -> my_udf}|my_custom_logger| + +-------+-----------------+---------------------+----------------+ + """ + return self._fn("python_worker_logs") + def _fn(self, functionName: str, *args: Column) -> DataFrame: from pyspark.sql.classic.column import _to_java_column diff --git a/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala b/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala index d2479ce9879c5..56f2f75c2c190 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala @@ -181,4 +181,12 @@ abstract class TableValuedFunction { * @since 4.0.0 */ def variant_explode_outer(input: Column): Dataset[Row] + + /** + * Returns a `DataFrame` of logs collected from Python workers. + * + * @group table_funcs + * @since 4.1.0 + */ + def python_worker_logs(): Dataset[Row] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index e6e713fab7d9b..36e40306be7d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.st._ import org.apache.spark.sql.catalyst.expressions.variant._ import org.apache.spark.sql.catalyst.expressions.xml._ -import org.apache.spark.sql.catalyst.plans.logical.{FunctionBuilderBase, Generate, LogicalPlan, OneRowRelation, Range} +import org.apache.spark.sql.catalyst.plans.logical.{FunctionBuilderBase, Generate, LogicalPlan, OneRowRelation, PythonWorkerLogs, Range} import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf @@ -1233,7 +1233,8 @@ object TableFunctionRegistry { generator[Collations]("collations"), generator[SQLKeywords]("sql_keywords"), generatorBuilder("variant_explode", VariantExplodeGeneratorBuilder), - generatorBuilder("variant_explode_outer", VariantExplodeOuterGeneratorBuilder) + generatorBuilder("variant_explode_outer", VariantExplodeOuterGeneratorBuilder), + PythonWorkerLogs.functionBuilder ) val builtin: SimpleTableFunctionRegistry = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala index ea5836b8ec2df..6ff40da88ed18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.catalog.{ TemporaryViewRelation, UnresolvedCatalogRelation } -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, PythonWorkerLogs, SubqueryAlias} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 import org.apache.spark.sql.connector.catalog.{ CatalogManager, @@ -112,8 +112,6 @@ class RelationResolution(override val catalogManager: CatalogManager) u.isStreaming, finalTimeTravelSpec.isDefined ).orElse { - resolveSystemSessionView(u.multipartIdentifier) - }.orElse { expandIdentifier(u.multipartIdentifier) match { case CatalogAndIdentifier(catalog, ident) => val key = toCacheKey(catalog, ident, finalTimeTravelSpec) @@ -238,22 +236,6 @@ class RelationResolution(override val catalogManager: CatalogManager) } } - private def isSystemSessionIdentifier(identifier: Seq[String]): Boolean = { - identifier.length > 2 && - identifier(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) && - identifier(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) - } - - private def resolveSystemSessionView( - identifier: Seq[String]): Option[LogicalPlan] = { - if (isSystemSessionIdentifier(identifier)) { - Option(identifier.drop(2)).collect { - case Seq(viewName) if viewName.equalsIgnoreCase(PythonWorkerLogs.ViewName) => - PythonWorkerLogs.viewDefinition() - } - } else None - } - private def toCacheKey( catalog: CatalogPlugin, ident: Identifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index 802678bc2c6ce..bcfcae2ee16c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -19,11 +19,13 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.resource.ResourceProfile import org.apache.spark.sql.catalyst.SQLConfHelper -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedAttribute, UnresolvedStar} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Expression, JsonToStructs, PythonUDF, PythonUDTF} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistryBase, MultiInstanceRelation, UnresolvedAttribute, UnresolvedStar} +import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Expression, ExpressionDescription, ExpressionInfo, JsonToStructs, PythonUDF, PythonUDTF} import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode, TimeMode} import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.LogUtils @@ -386,9 +388,26 @@ case class AttachDistributedSequence( } } +// scalastyle:off line.contains.tab line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_() - Returns a table of logs collected from Python workers. + """, + examples = """ + Examples: + > SET spark.sql.pyspark.worker.logging.enabled=true; + spark.sql.pyspark.worker.logging.enabled true + > SELECT * FROM _FUNC_(); + + """, + since = "4.1.0", + group = "table_funcs") +// scalastyle:on line.contains.tab line.size.limit case class PythonWorkerLogs(jsonAttr: Attribute) extends LeafNode with MultiInstanceRelation with SQLConfHelper { + def this() = this(DataTypeUtils.toAttribute(StructField("message", StringType))) + override def output: Seq[Attribute] = Seq(jsonAttr) override def newInstance(): PythonWorkerLogs = @@ -403,23 +422,28 @@ case class PythonWorkerLogs(jsonAttr: Attribute) ) } -object PythonWorkerLogs { - val ViewName = "python_worker_logs" - - def apply(): LogicalPlan = { - PythonWorkerLogs(DataTypeUtils.toAttribute(StructField("message", StringType))) - } - - def viewDefinition(): LogicalPlan = { - Project( - Seq(UnresolvedStar(Some(Seq("from_json")))), - Project( - Seq(Alias( - JsonToStructs( - schema = StructType.fromDDL(LogUtils.SPARK_LOG_SCHEMA), - options = Map.empty, - child = UnresolvedAttribute("message")), - "from_json")()), - PythonWorkerLogs())) +object PythonWorkerLogs extends SQLConfHelper { + val TableFunctionName = "python_worker_logs" + + val functionBuilder: (String, (ExpressionInfo, TableFunctionBuilder)) = { + val (info, builder) = FunctionRegistryBase.build[PythonWorkerLogs]( + TableFunctionName, None) + val funcBuilder = (expressions: Seq[Expression]) => { + if (conf.pythonWorkerLoggingEnabled) { + Project( + Seq(UnresolvedStar(Some(Seq("from_json")))), + Project( + Seq(Alias( + JsonToStructs( + schema = StructType.fromDDL(LogUtils.SPARK_LOG_SCHEMA), + options = Map.empty, + child = UnresolvedAttribute("message")), + "from_json")()), + builder(expressions))) + } else { + throw QueryCompilationErrors.pythonWorkerLoggingNotEnabledError() + } + } + TableFunctionName -> (info, funcBuilder) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 246973bbd246b..d00d639eb5de3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4419,4 +4419,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat origin = origin ) } + + def pythonWorkerLoggingNotEnabledError(): Throwable = { + new AnalysisException( + errorClass = "FEATURE_NOT_ENABLED", + messageParameters = Map( + "featureName" -> "Python Worker Logging", + "configKey" -> "spark.sql.pyspark.worker.logging.enabled", + "configValue" -> "true" + ) + ) + } } diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/TableValuedFunction.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/TableValuedFunction.scala index 05fc4b441f98e..e773f66cd6d05 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/TableValuedFunction.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/TableValuedFunction.scala @@ -102,4 +102,8 @@ class TableValuedFunction(sparkSession: SparkSession) extends sql.TableValuedFun /** @inheritdoc */ override def variant_explode_outer(input: Column): Dataset[Row] = fn("variant_explode_outer", Seq(input)) + + /** @inheritdoc */ + override def python_worker_logs(): Dataset[Row] = + fn("python_worker_logs", Seq.empty) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/TableValuedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/TableValuedFunction.scala index d2034033fee7e..839faa85aed48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/TableValuedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/TableValuedFunction.scala @@ -97,4 +97,8 @@ class TableValuedFunction(sparkSession: SparkSession) /** @inheritdoc */ override def variant_explode_outer(input: Column): Dataset[Row] = fn("variant_explode_outer", Seq(input)) + + /** @inheritdoc */ + override def python_worker_logs(): Dataset[Row] = + fn("python_worker_logs", Seq.empty) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonWorkerLogsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonWorkerLogsSuite.scala index 2f363d3e89540..ed195afc7e4d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonWorkerLogsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonWorkerLogsSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.execution.python import java.util.UUID -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.storage.{PythonWorkerLogBlockId, PythonWorkerLogBlockIdGenerator, PythonWorkerLogLine} @@ -28,6 +30,9 @@ import org.apache.spark.util.LogUtils class PythonWorkerLogsSuite extends QueryTest with SharedSparkSession { import testImplicits._ + override def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.PYTHON_WORKER_LOGGING_ENABLED.key, "true") + protected override def afterEach(): Unit = { try { val blockManager = spark.sparkContext.env.blockManager @@ -38,8 +43,25 @@ class PythonWorkerLogsSuite extends QueryTest with SharedSparkSession { } } + test("fail if logging is not enabled") { + withSQLConf(SQLConf.PYTHON_WORKER_LOGGING_ENABLED.key -> "false") { + val ex = intercept[AnalysisException] { + spark.tvf.python_worker_logs().collect() + } + checkError( + exception = ex, + condition = "FEATURE_NOT_ENABLED", + parameters = Map( + "featureName" -> "Python Worker Logging", + "configKey" -> "spark.sql.pyspark.worker.logging.enabled", + "configValue" -> "true" + ) + ) + } + } + test("schema") { - val schema = spark.table("system.session.python_worker_logs").schema + val schema = spark.tvf.python_worker_logs().schema assert(schema == StructType.fromDDL(LogUtils.SPARK_LOG_SCHEMA)) } @@ -57,7 +79,7 @@ class PythonWorkerLogsSuite extends QueryTest with SharedSparkSession { test("read logs") { prepareLogs(spark.sessionUUID) - val df = spark.table("system.session.python_worker_logs") + val df = spark.tvf.python_worker_logs() assert(df.count() == 2) checkAnswer( df.select($"level", $"msg"), @@ -67,7 +89,7 @@ class PythonWorkerLogsSuite extends QueryTest with SharedSparkSession { test("can't read logs for another session") { prepareLogs(UUID.randomUUID.toString) - val df = spark.table("system.session.python_worker_logs") + val df = spark.tvf.python_worker_logs() assert(df.count() == 0) } } From d65234b8578ce9a01ed656367646e2bea8f0f4fb Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Wed, 12 Nov 2025 18:06:15 -0800 Subject: [PATCH 121/400] [SPARK-54310][SQL] Add `numSourceRows` metric for `MergeIntoExec` ### What changes were proposed in this pull request? Add `numSourceRows` metric for `MergeIntoExec`, from source node's `numOutputRows`. Assumption is that all child nodes have `numOutputRows`. If not found, `numSourceRows` would be -1. ### Why are the changes needed? Improve completeness and debuggability of Merge Into metrics. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test cases for numSourceNodes metric. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52669 from asl3/merge-numsourcerows. Authored-by: Amanda Liu Signed-off-by: Dongjoon Hyun (cherry picked from commit 75fdec3d9a84be2729e6a70c0b8f1939b868f4c2) Signed-off-by: Dongjoon Hyun --- .../sql/connector/write/MergeSummary.java | 5 + .../connector/write/MergeSummaryImpl.scala | 1 + .../v2/WriteToDataSourceV2Exec.scala | 39 +- .../connector/MergeIntoTableSuiteBase.scala | 528 ++++++++++-------- 4 files changed, 346 insertions(+), 227 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/MergeSummary.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/MergeSummary.java index e5ae57a767080..37917bd7649de 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/MergeSummary.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/MergeSummary.java @@ -27,6 +27,11 @@ @Evolving public interface MergeSummary extends WriteSummary { + /** + * Returns the number of source rows. + */ + long numSourceRows(); + /** * Returns the number of target rows copied unmodified because they did not match any action, * or -1 if not found. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/MergeSummaryImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/MergeSummaryImpl.scala index 911749072c43c..f07f47061ee83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/MergeSummaryImpl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/MergeSummaryImpl.scala @@ -21,6 +21,7 @@ package org.apache.spark.sql.connector.write * Implementation of [[MergeSummary]] that provides MERGE operation summary. */ private[sql] case class MergeSummaryImpl( + numSourceRows: Long, numTargetRowsCopied: Long, numTargetRowsDeleted: Long, numTargetRowsUpdated: Long, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 7a2795b729f5a..1d7566ce7f3ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -31,10 +31,11 @@ import org.apache.spark.sql.catalyst.util.RowDeltaUtils.{DELETE_OPERATION, INSER import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, StagedTable, StagingTableCatalog, Table, TableCatalog, TableInfo, TableWritePrivilege} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.metric.CustomMetric -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, DeltaWrite, DeltaWriter, MergeSummaryImpl, PhysicalWriteInfoImpl, Write, WriterCommitMessage, WriteSummary} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, DeltaWrite, DeltaWriter, MergeSummaryImpl, PhysicalWriteInfoImpl, RowLevelOperationTable, Write, WriterCommitMessage, WriteSummary} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.{SparkPlan, SQLExecution, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.joins.BaseJoinExec import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric, SQLMetrics} import org.apache.spark.sql.types.StructType import org.apache.spark.util.{LongAccumulator, Utils} @@ -481,7 +482,9 @@ trait V2TableWriteExec extends V2CommandExec with UnaryExecNode with AdaptiveSpa private def getWriteSummary(query: SparkPlan): Option[WriteSummary] = { collectFirst(query) { case m: MergeRowsExec => m }.map { n => val metrics = n.metrics + val numSourceRows = getNumSourceRows(n) MergeSummaryImpl( + numSourceRows, metrics.get("numTargetRowsCopied").map(_.value).getOrElse(-1L), metrics.get("numTargetRowsDeleted").map(_.value).getOrElse(-1L), metrics.get("numTargetRowsUpdated").map(_.value).getOrElse(-1L), @@ -493,6 +496,40 @@ trait V2TableWriteExec extends V2CommandExec with UnaryExecNode with AdaptiveSpa ) } } + + private def getNumSourceRows(mergeRowsExec: MergeRowsExec): Long = { + def hasTargetTable(plan: SparkPlan): Boolean = { + collectFirst(plan) { + case scan @ BatchScanExec(_, _, _, _, _: RowLevelOperationTable, _) => scan + }.isDefined + } + + def findSourceScan(join: BaseJoinExec): Option[SparkPlan] = { + val leftHasTarget = hasTargetTable(join.left) + val rightHasTarget = hasTargetTable(join.right) + + val sourceSide = if (leftHasTarget) { + Some(join.right) + } else if (rightHasTarget) { + Some(join.left) + } else { + None + } + + sourceSide.flatMap { side => + collectFirst(side) { + case source if source.metrics.contains("numOutputRows") => + source + } + } + } + + (for { + join <- collectFirst(mergeRowsExec.child) { case j: BaseJoinExec => j } + sourceScan <- findSourceScan(join) + metric <- sourceScan.metrics.get("numOutputRows") + } yield metric.value).getOrElse(-1L) + } } trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index b73c8d2458c87..85b0faed4c388 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -1778,159 +1778,179 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } test("Merge metrics with matched clause") { - withTempView("source") { - createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", - """{ "pk": 1, "salary": 100, "dep": "hr" } - |{ "pk": 2, "salary": 200, "dep": "software" } - |{ "pk": 3, "salary": 300, "dep": "hr" } - |""".stripMargin) + Seq("true", "false").foreach { aqeEnabled: String => + withTempView("source") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |""".stripMargin) - val sourceDF = Seq(1, 2, 10).toDF("pk") - sourceDF.createOrReplaceTempView("source") + val sourceDF = Seq(1, 2, 10).toDF("pk") + sourceDF.createOrReplaceTempView("source") - val mergeExec = findMergeExec { - s"""MERGE INTO $tableNameAsString t - |USING source s - |ON t.pk = s.pk - |WHEN MATCHED AND salary < 200 THEN - | UPDATE SET salary = 1000 - |""".stripMargin - } + val mergeExec = findMergeExec { + s"""MERGE INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED AND salary < 200 THEN + | UPDATE SET salary = 1000 + |""".stripMargin + } - assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 2) - assertMetric(mergeExec, "numTargetRowsInserted", 0) - assertMetric(mergeExec, "numTargetRowsUpdated", 1) - assertMetric(mergeExec, "numTargetRowsDeleted", 0) - assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 1) - assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) + assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 2) + assertMetric(mergeExec, "numTargetRowsInserted", 0) + assertMetric(mergeExec, "numTargetRowsUpdated", 1) + assertMetric(mergeExec, "numTargetRowsDeleted", 0) + assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 1) + assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, 1000, "hr"), // updated - Row(2, 200, "software"), - Row(3, 300, "hr"))) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 1000, "hr"), // updated + Row(2, 200, "software"), + Row(3, 300, "hr"))) + } - val mergeSummary = getMergeSummary() - assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 2L)) - assert(mergeSummary.numTargetRowsInserted === 0L) - assert(mergeSummary.numTargetRowsUpdated === 1L) - assert(mergeSummary.numTargetRowsDeleted === 0L) - assert(mergeSummary.numTargetRowsMatchedUpdated === 1L) - assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + val mergeSummary = getMergeSummary() + assert(mergeSummary.numSourceRows === 3L) + assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 2L)) + assert(mergeSummary.numTargetRowsInserted === 0L) + assert(mergeSummary.numTargetRowsUpdated === 1L) + assert(mergeSummary.numTargetRowsDeleted === 0L) + assert(mergeSummary.numTargetRowsMatchedUpdated === 1L) + assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + + sql(s"DROP TABLE $tableNameAsString") + } } } test("Merge metrics with matched and not matched clause") { - withTempView("source") { - createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", - """{ "pk": 1, "salary": 100, "dep": "hr" } - |{ "pk": 2, "salary": 200, "dep": "software" } - |{ "pk": 3, "salary": 300, "dep": "hr" } - |""".stripMargin) + Seq("true", "false").foreach { aqeEnabled: String => + withTempView("source") { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |""".stripMargin) - val sourceDF = Seq( - (4, 100, "marketing"), - (5, 400, "executive"), - (6, 100, "hr") - ).toDF("pk", "salary", "dep") - sourceDF.createOrReplaceTempView("source") + val sourceDF = Seq( + (4, 100, "marketing"), + (5, 400, "executive"), + (6, 100, "hr") + ).toDF("pk", "salary", "dep") + sourceDF.createOrReplaceTempView("source") - val mergeExec = findMergeExec { - s"""MERGE INTO $tableNameAsString t - |USING source s - |ON t.pk = s.pk - |WHEN MATCHED THEN - | UPDATE SET salary = 9999 - |WHEN NOT MATCHED AND salary > 200 THEN - | INSERT * - |""".stripMargin - } + val mergeExec = findMergeExec { + s"""MERGE INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET salary = 9999 + |WHEN NOT MATCHED AND salary > 200 THEN + | INSERT * + |""".stripMargin + } - assertMetric(mergeExec, "numTargetRowsCopied", 0) - assertMetric(mergeExec, "numTargetRowsInserted", 1) - assertMetric(mergeExec, "numTargetRowsUpdated", 0) - assertMetric(mergeExec, "numTargetRowsDeleted", 0) - assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 0) - assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) + assertMetric(mergeExec, "numTargetRowsCopied", 0) + assertMetric(mergeExec, "numTargetRowsInserted", 1) + assertMetric(mergeExec, "numTargetRowsUpdated", 0) + assertMetric(mergeExec, "numTargetRowsDeleted", 0) + assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 0) + assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, 100, "hr"), - Row(2, 200, "software"), - Row(3, 300, "hr"), - Row(5, 400, "executive"))) // inserted + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr"), + Row(2, 200, "software"), + Row(3, 300, "hr"), + Row(5, 400, "executive"))) // inserted + + val mergeSummary = getMergeSummary() + // TODO SPARK-52578: Handle this case when optimizer removes Join due to no matching pks + assert(mergeSummary.numSourceRows === (if (deltaMerge) 3L else -1L)) + assert(mergeSummary.numTargetRowsCopied === 0L) + assert(mergeSummary.numTargetRowsInserted === 1L) + assert(mergeSummary.numTargetRowsUpdated === 0L) + assert(mergeSummary.numTargetRowsDeleted === 0L) + assert(mergeSummary.numTargetRowsMatchedUpdated === 0L) + assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) - val mergeSummary = getMergeSummary() - assert(mergeSummary.numTargetRowsCopied === 0L) - assert(mergeSummary.numTargetRowsInserted === 1L) - assert(mergeSummary.numTargetRowsUpdated === 0L) - assert(mergeSummary.numTargetRowsDeleted === 0L) - assert(mergeSummary.numTargetRowsMatchedUpdated === 0L) - assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + sql(s"DROP TABLE $tableNameAsString") + } } } test("Merge metrics with matched and not matched by source clauses: update") { - withTempView("source") { - createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", - """{ "pk": 1, "salary": 100, "dep": "hr" } - |{ "pk": 2, "salary": 200, "dep": "software" } - |{ "pk": 3, "salary": 300, "dep": "hr" } - |{ "pk": 4, "salary": 400, "dep": "marketing" } - |{ "pk": 5, "salary": 500, "dep": "executive" } - |""".stripMargin) + Seq("true", "false").foreach { aqeEnabled: String => + withTempView("source") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |{ "pk": 4, "salary": 400, "dep": "marketing" } + |{ "pk": 5, "salary": 500, "dep": "executive" } + |""".stripMargin) - val sourceDF = Seq(1, 2, 10).toDF("pk") - sourceDF.createOrReplaceTempView("source") + val sourceDF = Seq(1, 2, 10).toDF("pk") + sourceDF.createOrReplaceTempView("source") - val mergeExec = findMergeExec { - s"""MERGE INTO $tableNameAsString t - |USING source s - |ON t.pk = s.pk - |WHEN MATCHED AND salary < 200 THEN - | UPDATE SET salary = 1000 - |WHEN NOT MATCHED BY SOURCE AND salary > 400 THEN - | UPDATE SET salary = -1 - |""".stripMargin - } + val mergeExec = findMergeExec { + s"""MERGE INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED AND salary < 200 THEN + | UPDATE SET salary = 1000 + |WHEN NOT MATCHED BY SOURCE AND salary > 400 THEN + | UPDATE SET salary = -1 + |""".stripMargin + } - assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 3) - assertMetric(mergeExec, "numTargetRowsInserted", 0) - assertMetric(mergeExec, "numTargetRowsUpdated", 2) - assertMetric(mergeExec, "numTargetRowsDeleted", 0) - assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 1) - assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 1) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) + assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 3) + assertMetric(mergeExec, "numTargetRowsInserted", 0) + assertMetric(mergeExec, "numTargetRowsUpdated", 2) + assertMetric(mergeExec, "numTargetRowsDeleted", 0) + assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 1) + assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 1) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, 1000, "hr"), // updated - Row(2, 200, "software"), - Row(3, 300, "hr"), - Row(4, 400, "marketing"), - Row(5, -1, "executive"))) // updated + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 1000, "hr"), // updated + Row(2, 200, "software"), + Row(3, 300, "hr"), + Row(4, 400, "marketing"), + Row(5, -1, "executive"))) // updated + } - val mergeSummary = getMergeSummary() - assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) - assert(mergeSummary.numTargetRowsInserted === 0L) - assert(mergeSummary.numTargetRowsUpdated === 2L) - assert(mergeSummary.numTargetRowsDeleted === 0L) - assert(mergeSummary.numTargetRowsMatchedUpdated === 1L) - assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 1L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + val mergeSummary = getMergeSummary() + assert(mergeSummary.numSourceRows === 3L) + assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) + assert(mergeSummary.numTargetRowsInserted === 0L) + assert(mergeSummary.numTargetRowsUpdated === 2L) + assert(mergeSummary.numTargetRowsDeleted === 0L) + assert(mergeSummary.numTargetRowsMatchedUpdated === 1L) + assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 1L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + + sql(s"DROP TABLE $tableNameAsString") + } } } @@ -1979,6 +1999,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase ) val mergeSummary = getMergeSummary() + assert(mergeSummary.numSourceRows === 3L) assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) assert(mergeSummary.numTargetRowsInserted === 0L) assert(mergeSummary.numTargetRowsUpdated === 0L) @@ -1991,116 +2012,130 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } test("Merge metrics with matched, not matched, and not matched by source clauses: update") { - withTempView("source") { - createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", - """{ "pk": 1, "salary": 100, "dep": "hr" } - |{ "pk": 2, "salary": 200, "dep": "software" } - |{ "pk": 3, "salary": 300, "dep": "hr" } - |{ "pk": 4, "salary": 400, "dep": "marketing" } - |{ "pk": 5, "salary": 500, "dep": "executive" } - |""".stripMargin) + Seq("true", "false").foreach { aqeEnabled: String => + withTempView("source") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |{ "pk": 4, "salary": 400, "dep": "marketing" } + |{ "pk": 5, "salary": 500, "dep": "executive" } + |""".stripMargin) - val sourceDF = Seq(1, 2, 6, 10).toDF("pk") - sourceDF.createOrReplaceTempView("source") + val sourceDF = Seq(1, 2, 6, 10).toDF("pk") + sourceDF.createOrReplaceTempView("source") - val mergeExec = findMergeExec { - s"""MERGE INTO $tableNameAsString t - |USING source s - |ON t.pk = s.pk - |WHEN MATCHED AND salary < 200 THEN - | UPDATE SET salary = 1000 - |WHEN NOT MATCHED AND s.pk < 10 THEN - | INSERT (pk, salary, dep) VALUES (s.pk, -1, "dummy") - |WHEN NOT MATCHED BY SOURCE AND salary > 400 THEN - | UPDATE SET salary = -1 - |""".stripMargin - } + val mergeExec = findMergeExec { + s"""MERGE INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED AND salary < 200 THEN + | UPDATE SET salary = 1000 + |WHEN NOT MATCHED AND s.pk < 10 THEN + | INSERT (pk, salary, dep) VALUES (s.pk, -1, "dummy") + |WHEN NOT MATCHED BY SOURCE AND salary > 400 THEN + | UPDATE SET salary = -1 + |""".stripMargin + } - assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 3) - assertMetric(mergeExec, "numTargetRowsInserted", 1) - assertMetric(mergeExec, "numTargetRowsUpdated", 2) - assertMetric(mergeExec, "numTargetRowsDeleted", 0) - assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 1) - assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 1) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) + assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 3) + assertMetric(mergeExec, "numTargetRowsInserted", 1) + assertMetric(mergeExec, "numTargetRowsUpdated", 2) + assertMetric(mergeExec, "numTargetRowsDeleted", 0) + assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 1) + assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 1) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 0) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, 1000, "hr"), // updated - Row(2, 200, "software"), - Row(3, 300, "hr"), - Row(4, 400, "marketing"), - Row(5, -1, "executive"), // updated - Row(6, -1, "dummy"))) // inserted + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 1000, "hr"), // updated + Row(2, 200, "software"), + Row(3, 300, "hr"), + Row(4, 400, "marketing"), + Row(5, -1, "executive"), // updated + Row(6, -1, "dummy"))) // inserted + } - val mergeSummary = getMergeSummary() - assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) - assert(mergeSummary.numTargetRowsInserted === 1L) - assert(mergeSummary.numTargetRowsUpdated === 2L) - assert(mergeSummary.numTargetRowsDeleted === 0L) - assert(mergeSummary.numTargetRowsMatchedUpdated === 1L) - assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 1L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + val mergeSummary = getMergeSummary() + assert(mergeSummary.numSourceRows === 4L) + assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) + assert(mergeSummary.numTargetRowsInserted === 1L) + assert(mergeSummary.numTargetRowsUpdated === 2L) + assert(mergeSummary.numTargetRowsDeleted === 0L) + assert(mergeSummary.numTargetRowsMatchedUpdated === 1L) + assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 1L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 0L) + + sql(s"DROP TABLE $tableNameAsString") + } } } test("Merge metrics with matched, not matched, and not matched by source clauses: delete") { - withTempView("source") { - createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", - """{ "pk": 1, "salary": 100, "dep": "hr" } - |{ "pk": 2, "salary": 200, "dep": "software" } - |{ "pk": 3, "salary": 300, "dep": "hr" } - |{ "pk": 4, "salary": 400, "dep": "marketing" } - |{ "pk": 5, "salary": 500, "dep": "executive" } - |""".stripMargin) + Seq("true", "false").foreach { aqeEnabled: String => + withTempView("source") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |{ "pk": 4, "salary": 400, "dep": "marketing" } + |{ "pk": 5, "salary": 500, "dep": "executive" } + |""".stripMargin) - val sourceDF = Seq(1, 2, 6, 10).toDF("pk") - sourceDF.createOrReplaceTempView("source") + val sourceDF = Seq(1, 2, 6, 10).toDF("pk") + sourceDF.createOrReplaceTempView("source") - val mergeExec = findMergeExec { - s"""MERGE INTO $tableNameAsString t - |USING source s - |ON t.pk = s.pk - |WHEN MATCHED AND salary < 200 THEN - | DELETE - |WHEN NOT MATCHED AND s.pk < 10 THEN - | INSERT (pk, salary, dep) VALUES (s.pk, -1, "dummy") - |WHEN NOT MATCHED BY SOURCE AND salary > 400 THEN - | DELETE - |""".stripMargin - } + val mergeExec = findMergeExec { + s"""MERGE INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED AND salary < 200 THEN + | DELETE + |WHEN NOT MATCHED AND s.pk < 10 THEN + | INSERT (pk, salary, dep) VALUES (s.pk, -1, "dummy") + |WHEN NOT MATCHED BY SOURCE AND salary > 400 THEN + | DELETE + |""".stripMargin + } - assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 3) - assertMetric(mergeExec, "numTargetRowsInserted", 1) - assertMetric(mergeExec, "numTargetRowsUpdated", 0) - assertMetric(mergeExec, "numTargetRowsDeleted", 2) - assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 0) - assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 1) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 0) - assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 1) + assertMetric(mergeExec, "numTargetRowsCopied", if (deltaMerge) 0 else 3) + assertMetric(mergeExec, "numTargetRowsInserted", 1) + assertMetric(mergeExec, "numTargetRowsUpdated", 0) + assertMetric(mergeExec, "numTargetRowsDeleted", 2) + assertMetric(mergeExec, "numTargetRowsMatchedUpdated", 0) + assertMetric(mergeExec, "numTargetRowsMatchedDeleted", 1) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceUpdated", 0) + assertMetric(mergeExec, "numTargetRowsNotMatchedBySourceDeleted", 1) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - // Row(1, 100, "hr") deleted - Row(2, 200, "software"), - Row(3, 300, "hr"), - Row(4, 400, "marketing"), - // Row(5, 500, "executive") deleted - Row(6, -1, "dummy"))) // inserted + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + // Row(1, 100, "hr") deleted + Row(2, 200, "software"), + Row(3, 300, "hr"), + Row(4, 400, "marketing"), + // Row(5, 500, "executive") deleted + Row(6, -1, "dummy"))) // inserted + } - val mergeSummary = getMergeSummary() - assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) - assert(mergeSummary.numTargetRowsInserted === 1L) - assert(mergeSummary.numTargetRowsUpdated === 0L) - assert(mergeSummary.numTargetRowsDeleted === 2L) - assert(mergeSummary.numTargetRowsMatchedUpdated === 0L) - assert(mergeSummary.numTargetRowsMatchedDeleted === 1L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) - assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 1L) + val mergeSummary = getMergeSummary() + assert(mergeSummary.numSourceRows === 4L) + assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) + assert(mergeSummary.numTargetRowsInserted === 1L) + assert(mergeSummary.numTargetRowsUpdated === 0L) + assert(mergeSummary.numTargetRowsDeleted === 2L) + assert(mergeSummary.numTargetRowsMatchedUpdated === 0L) + assert(mergeSummary.numTargetRowsMatchedDeleted === 1L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 1L) + + sql(s"DROP TABLE $tableNameAsString") + } } } @@ -2133,6 +2168,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase ) val mergeMetrics = getMergeSummary() + assert(mergeMetrics.numSourceRows === 4L) assert(mergeMetrics.numTargetRowsCopied === (if (deltaMerge) 0L else 3L)) assert(mergeMetrics.numTargetRowsInserted === 1L) assert(mergeMetrics.numTargetRowsUpdated === 0L) @@ -2148,6 +2184,46 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } + test("Merge metrics with numSourceRows for empty source") { + Seq("true", "false").foreach { aqeEnabled: String => + withTempView("source") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |{ "pk": 3, "salary": 300, "dep": "hr" } + |""".stripMargin) + + // source is empty + Seq.empty[Int].toDF("pk").createOrReplaceTempView("source") + + sql(s"""MERGE INTO $tableNameAsString t + |USING source s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET salary = 1000 + |WHEN NOT MATCHED BY SOURCE THEN + | DELETE + |""".stripMargin) + + val mergeSummary = getMergeSummary() + assert(mergeSummary.numSourceRows === -1L) // if no numOutputRows, should be -1 + assert(mergeSummary.numTargetRowsCopied === (if (deltaMerge) 0L else 0L)) + assert(mergeSummary.numTargetRowsInserted === 0L) + assert(mergeSummary.numTargetRowsUpdated === 0L) + assert(mergeSummary.numTargetRowsDeleted === 3L) + assert(mergeSummary.numTargetRowsMatchedUpdated === 0L) + assert(mergeSummary.numTargetRowsMatchedDeleted === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceUpdated === 0L) + assert(mergeSummary.numTargetRowsNotMatchedBySourceDeleted === 3L) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + } + test("Merge schema evolution new column with set explicit column") { Seq((true, true), (false, true), (true, false)).foreach { case (withSchemaEvolution, schemaEvolutionEnabled) => From 904219853b77cd27e07cc6b49d4b231a2b0102f1 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 13 Nov 2025 10:29:10 +0800 Subject: [PATCH 122/400] [SPARK-54315][PYTHON][TESTS] Optimize test `ApplyInArrowTests.test_arrow_batch_slicing` ### What changes were proposed in this pull request? Optimize test `ApplyInArrowTests.test_arrow_batch_slicing` ### Why are the changes needed? before ``` Starting test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map (temp output: /__w/spark/spark/python/target/84531e82-addd-4a47-bb65-9006099020a0/python3.11__pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map__l8scrw6b.log) Finished test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map (100s) ``` after ``` Starting test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map (temp output: /__w/spark/spark/python/target/f8e27057-d295-433e-99a2-a429a48278d5/python3.11__pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map__ss4itrd9.log) Finished test(python3.11): pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map (39s) ``` ### Does this PR introduce _any_ user-facing change? No, test-only ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #53011 from zhengruifeng/opt_test_grouped. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit 65b69a0d9fd58378eb46ed3e368925d228039b6c) Signed-off-by: Ruifeng Zheng --- .../pyspark/sql/tests/arrow/test_arrow_grouped_map.py | 10 +++++----- .../sql/tests/pandas/test_pandas_grouped_map.py | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py b/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py index 56a9ed322c154..cb8d74f724270 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py @@ -356,14 +356,14 @@ def arrow_func(key, table): self.assertEqual(df2.join(df2).count(), 1) def test_arrow_batch_slicing(self): - df = self.spark.range(10000000).select( - (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") - ) + n = 100000 + + df = self.spark.range(n).select((sf.col("id") % 2).alias("key"), sf.col("id").alias("v")) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} df = df.withColumns(cols) def min_max_v(table): - assert len(table) == 10000000 / 2, len(table) + assert len(table) == n / 2, len(table) return pa.Table.from_pydict( { "key": [table.column("key")[0].as_py()], @@ -376,7 +376,7 @@ def min_max_v(table): df.groupby("key").agg(sf.min("v").alias("min"), sf.max("v").alias("max")).sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 1048576), (1000, 1048576)]: + for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index 8bf0e7235efcf..a86ef3e4f3f32 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -938,14 +938,14 @@ def test(pdf): self.assertEqual(row[1], 123) def test_arrow_batch_slicing(self): - df = self.spark.range(100000).select( - (sf.col("id") % 2).alias("key"), sf.col("id").alias("v") - ) + n = 100000 + + df = self.spark.range(n).select((sf.col("id") % 2).alias("key"), sf.col("id").alias("v")) cols = {f"col_{i}": sf.col("v") + i for i in range(20)} df = df.withColumns(cols) def min_max_v(pdf): - assert len(pdf) == 100000 / 2, len(pdf) + assert len(pdf) == n / 2, len(pdf) return pd.DataFrame( { "key": [pdf.key.iloc[0]], @@ -958,7 +958,7 @@ def min_max_v(pdf): df.groupby("key").agg(sf.min("v").alias("min"), sf.max("v").alias("max")).sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 4096), (0, 4096), (1000, 4096)]: + for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { From 71559b31c2c457c7d3cc091424260ff0c8b87fba Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 12 Nov 2025 18:40:24 -0800 Subject: [PATCH 123/400] [SPARK-54326][INFRA] Recover MacOS CIs by installing `zstandard==0.25.0` ### What changes were proposed in this pull request? This PR aims to recover MacOS CIs by installing `zstandard==0.25.0`. ### Why are the changes needed? After SPARK-54194, `zstandard` Python package is required for `Connect`. - https://github.com/apache/spark/pull/52894 Currently, MacOS CIs are broken. - https://github.com/apache/spark/actions/workflows/build_python_3.11_macos26.yml - https://github.com/apache/spark/actions/workflows/build_python_3.11_macos.yml ``` Traceback (most recent call last): File "/Users/runner/work/spark/spark/python/pyspark/sql/connect/utils.py", line 105, in require_minimum_zstandard_version import zstandard # noqa ^^^^^^^^^^^^^^^^ ModuleNotFoundError: No module named 'zstandard' ``` ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs and manual review because MacOS CIs are triggered daily CIs only. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53025 from dongjoon-hyun/SPARK-54326. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 5c16a73c61c3174beaa6ee19c1b0884e4e64d423) Signed-off-by: Dongjoon Hyun --- .github/workflows/python_hosted_runner_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index b7ee3a8b7113c..c0fe016326d49 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -148,7 +148,7 @@ jobs: python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' python${{matrix.python}} -m pip install numpy 'pyarrow>=22.0.0' 'six==1.16.0' 'pandas==2.3.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ - python${{matrix.python}} -m pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'graphviz==0.20.3' && \ + python${{matrix.python}} -m pip install 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'googleapis-common-protos==1.71.0' 'zstandard==0.25.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge - name: List Python packages run: python${{matrix.python}} -m pip list From 63205400f9bc861e2bfd0a876e5061e2809b4f55 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Wed, 12 Nov 2025 19:47:21 -0800 Subject: [PATCH 124/400] [SPARK-53573][SQL] IDENTIFIER everywhere ### What changes were proposed in this pull request? We propose expanding the IDENTIFIER() clause, which turns a string into a qualified identifier, to all places identifiers can appear. The current clause is severely limited in where it can go because it accepts constant expressions, including session variables. Due to the complexity of the argument the existing clause requires tricky code to incrementally analyze its arguments and then execute sections of parser code at a later point. By contrast the generalized IDENTIFIER clause only allows string literals which can be processed in the visitor methods. Due to the rework of parameter markers and string coalescing this allows for constructs such as: ``` SELECT * FROM IDENTIFIER(:cat '.' :schema '.' :table) ``` it even allows: ``` SELECT 'hello' AS IDENTIFIER(:alias); ``` This is really all identifier() needs. We may be able to deprecate and de-support the existing too complex identifier() implementation. ### Why are the changes needed? IDENTIFIER() is a popular feature, but it can only be used in very specific, hard to reason about places. The new implementation preserved 99% of teh fucntion while expanding its use to everywhere. ### Does this PR introduce _any_ user-facing change? Yes, it's a new feature ### How was this patch tested? expanded Parameters and inentifier-clause test suites. ### Was this patch authored or co-authored using generative AI tooling? Yes, Clause Sonnet 4.5 Closes #52765 from srielau/identifier-lite. Authored-by: Serge Rielau Signed-off-by: Wenchen Fan (cherry picked from commit daa29f6ede29f8a705c2c757b66a3471b80bb1ec) Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 7 +- .../sql/catalyst/parser/SqlBaseParser.g4 | 48 +- .../catalyst/parser/DataTypeAstBuilder.scala | 130 +- .../parser/SubstituteParmsAstBuilder.scala | 6 +- .../spark/sql/catalyst/parser/parsers.scala | 11 +- .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../spark/sql/internal/SqlApiConf.scala | 2 + .../sql/catalyst/parser/AstBuilder.scala | 122 +- .../sql/catalyst/parser/ParserUtils.scala | 79 +- .../parser/SubstituteParamsParser.scala | 18 +- .../sql/errors/QueryCompilationErrors.scala | 8 +- .../sql/errors/QueryExecutionErrors.scala | 8 +- .../apache/spark/sql/internal/SQLConf.scala | 14 + .../parser/SqlScriptingParserSuite.scala | 18 +- .../spark/sql/classic/SparkSession.scala | 87 +- .../spark/sql/execution/SparkSqlParser.scala | 15 +- .../spark/sql/execution/SparkStrategies.scala | 33 +- .../execute-immediate.sql.out | 40 + .../identifier-clause-legacy.sql.out | 2431 +++++++++++++++ .../identifier-clause.sql.out | 1079 ++++++- .../sql-tests/inputs/execute-immediate.sql | 8 + .../inputs/identifier-clause-legacy.sql | 2 + .../sql-tests/inputs/identifier-clause.sql | 209 +- .../results/execute-immediate.sql.out | 44 + .../results/identifier-clause-legacy.sql.out | 2736 +++++++++++++++++ .../results/identifier-clause.sql.out | 1157 ++++++- .../apache/spark/sql/ParametersSuite.scala | 72 + .../sql/StringLiteralCoalescingSuite.scala | 37 + .../sql/connector/DataSourceV2SQLSuite.scala | 12 +- .../errors/QueryCompilationErrorsSuite.scala | 4 +- .../v2/V2SessionCatalogSuite.scala | 4 +- .../apache/spark/sql/jdbc/JDBCV2Suite.scala | 2 +- .../sql/scripting/SqlScriptingE2eSuite.scala | 62 + 33 files changed, 8155 insertions(+), 352 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index ced4d70ab8ddc..489f3fbad34ec 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2059,7 +2059,7 @@ }, "IDENTIFIER_TOO_MANY_NAME_PARTS" : { "message" : [ - " is not a valid identifier as it has more than 2 name parts." + " is not a valid identifier as it has more than name parts." ], "sqlState" : "42601" }, @@ -8624,11 +8624,6 @@ "Failed to merge incompatible schemas and ." ] }, - "_LEGACY_ERROR_TEMP_2096" : { - "message" : [ - " is not supported temporarily." - ] - }, "_LEGACY_ERROR_TEMP_2097" : { "message" : [ "Could not execute broadcast in secs. You can increase the timeout for broadcasts via or disable broadcast join by setting to -1 or remove the broadcast hint if it exists in your code." diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 8ccac6a39d2ce..9d942bc601592 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -46,6 +46,13 @@ options { tokenVocab = SqlBaseLexer; } * When true, parameter markers are allowed everywhere a literal is supported. */ public boolean parameter_substitution_enabled = true; + + /** + * When false (default), IDENTIFIER('literal') is resolved to an identifier at parse time (identifier-lite). + * When true, only the legacy IDENTIFIER(expression) function syntax is allowed. + * Controlled by spark.sql.legacy.identifierClause configuration. + */ + public boolean legacy_identifier_clause_only = false; } compoundOrSingleStatement @@ -92,7 +99,7 @@ sqlStateValue ; declareConditionStatement - : DECLARE multipartIdentifier CONDITION (FOR SQLSTATE VALUE? sqlStateValue)? + : DECLARE strictIdentifier CONDITION (FOR SQLSTATE VALUE? sqlStateValue)? ; conditionValue @@ -125,11 +132,11 @@ repeatStatement ; leaveStatement - : LEAVE multipartIdentifier + : LEAVE strictIdentifier ; iterateStatement - : ITERATE multipartIdentifier + : ITERATE strictIdentifier ; caseStatement @@ -144,7 +151,7 @@ loopStatement ; forStatement - : beginLabel? FOR (multipartIdentifier AS)? query DO compoundBody END FOR endLabel? + : beginLabel? FOR (strictIdentifier AS)? query DO compoundBody END FOR endLabel? ; singleStatement @@ -152,11 +159,11 @@ singleStatement ; beginLabel - : multipartIdentifier COLON + : strictIdentifier COLON ; endLabel - : multipartIdentifier + : strictIdentifier ; singleExpression @@ -321,7 +328,7 @@ statement | SHOW VIEWS ((FROM | IN) identifierReference)? (LIKE? pattern=stringLit)? #showViews | SHOW PARTITIONS identifierReference partitionSpec? #showPartitions - | SHOW identifier? FUNCTIONS ((FROM | IN) ns=identifierReference)? + | SHOW functionScope=simpleIdentifier? FUNCTIONS ((FROM | IN) ns=identifierReference)? (LIKE? (legacy=multipartIdentifier | pattern=stringLit))? #showFunctions | SHOW PROCEDURES ((FROM | IN) identifierReference)? #showProcedures | SHOW CREATE TABLE identifierReference (AS SERDE)? #showCreateTable @@ -833,8 +840,8 @@ hint ; hintStatement - : hintName=identifier - | hintName=identifier LEFT_PAREN parameters+=primaryExpression (COMMA parameters+=primaryExpression)* RIGHT_PAREN + : hintName=simpleIdentifier + | hintName=simpleIdentifier LEFT_PAREN parameters+=primaryExpression (COMMA parameters+=primaryExpression)* RIGHT_PAREN ; fromClause @@ -1241,7 +1248,7 @@ primaryExpression | identifier #columnReference | base=primaryExpression DOT fieldName=identifier #dereference | LEFT_PAREN expression RIGHT_PAREN #parenthesizedExpression - | EXTRACT LEFT_PAREN field=identifier FROM source=valueExpression RIGHT_PAREN #extract + | EXTRACT LEFT_PAREN field=simpleIdentifier FROM source=valueExpression RIGHT_PAREN #extract | (SUBSTR | SUBSTRING) LEFT_PAREN str=valueExpression (FROM | COMMA) pos=valueExpression ((FOR | COMMA) len=valueExpression)? RIGHT_PAREN #substring | TRIM LEFT_PAREN trimOption=(BOTH | LEADING | TRAILING)? (trimStr=valueExpression)? @@ -1297,7 +1304,7 @@ constant ; namedParameterMarker - : COLON identifier + : COLON simpleIdentifier ; comparisonOperator : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ @@ -1599,13 +1606,32 @@ identifier | {!SQL_standard_keyword_behavior}? strictNonReserved ; +// simpleIdentifier: like identifier but without IDENTIFIER('literal') support +// Use this for contexts where IDENTIFIER() syntax is not appropriate: +// - Named parameters (:param_name) +// - Extract field names (EXTRACT(field FROM ...)) +// - Other keyword-like or string-like uses +simpleIdentifier + : simpleStrictIdentifier + | {!SQL_standard_keyword_behavior}? strictNonReserved + ; + strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative + | {!legacy_identifier_clause_only}? IDENTIFIER_KW LEFT_PAREN stringLit RIGHT_PAREN #identifierLiteral | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier ; +// simpleStrictIdentifier: like strictIdentifier but without IDENTIFIER('literal') support +simpleStrictIdentifier + : IDENTIFIER #simpleUnquotedIdentifier + | quotedIdentifier #simpleQuotedIdentifierAlternative + | {SQL_standard_keyword_behavior}? ansiNonReserved #simpleUnquotedIdentifier + | {!SQL_standard_keyword_behavior}? nonReserved #simpleUnquotedIdentifier + ; + quotedIdentifier : BACKQUOTED_IDENTIFIER | {double_quoted_identifiers}? DOUBLEQUOTED_STRING diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index 73767990bd3a3..212c80a3cb435 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -20,7 +20,7 @@ import java.util.Locale import scala.jdk.CollectionConverters._ -import org.antlr.v4.runtime.Token +import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.ParseTree import org.apache.spark.SparkException @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.catalyst.util.SparkParserUtils.{string, withOrigin} import org.apache.spark.sql.connector.catalog.IdentityColumnSpec -import org.apache.spark.sql.errors.QueryParsingErrors +import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryParsingErrors} import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, CalendarIntervalType, CharType, DataType, DateType, DayTimeIntervalType, DecimalType, DoubleType, FloatType, GeographyType, GeometryType, IntegerType, LongType, MapType, MetadataBuilder, NullType, ShortType, StringType, StructField, StructType, TimestampNTZType, TimestampType, TimeType, VarcharType, VariantType, YearMonthIntervalType} @@ -60,12 +60,52 @@ import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, * * @see * [[org.apache.spark.sql.catalyst.parser.AstBuilder]] for the full SQL statement parser + * + * ==CRITICAL: Extracting Identifier Names== + * + * When extracting identifier names from parser contexts, you MUST use the helper methods provided + * by this class instead of calling ctx.getText() directly: + * + * - '''getIdentifierText(ctx)''': For single identifiers (column names, aliases, window names) + * - '''getIdentifierParts(ctx)''': For qualified identifiers (table names, schema.table) + * + * '''DO NOT use ctx.getText() or ctx.identifier.getText()''' directly! These methods do not + * handle the IDENTIFIER('literal') syntax and will cause incorrect behavior. + * + * The IDENTIFIER('literal') syntax allows string literals to be used as identifiers at parse time + * (e.g., IDENTIFIER('my_col') resolves to the identifier my_col). If you use getText(), you'll + * get the raw text "IDENTIFIER('my_col')" instead of "my_col", breaking the feature. + * + * Example: + * {{{ + * // WRONG - does not handle IDENTIFIER('literal'): + * val name = ctx.identifier.getText + * SubqueryAlias(ctx.name.getText, plan) + * + * // CORRECT - handles both regular identifiers and IDENTIFIER('literal'): + * val name = getIdentifierText(ctx.identifier) + * SubqueryAlias(getIdentifierText(ctx.name), plan) + * }}} */ -class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { +class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with DataTypeErrorsBase { protected def typedVisit[T](ctx: ParseTree): T = { ctx.accept(this).asInstanceOf[T] } + /** + * Public helper to extract identifier parts from a context. This is exposed as public to allow + * utility classes like ParserUtils to reuse the identifier resolution logic without duplicating + * code. + * + * @param ctx + * The parser context containing the identifier. + * @return + * Sequence of identifier parts. + */ + def extractIdentifierParts(ctx: ParserRuleContext): Seq[String] = { + getIdentifierParts(ctx) + } + override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) { typedVisit[DataType](ctx.dataType) } @@ -161,11 +201,89 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { } /** - * Create a multi-part identifier. + * Parse a string into a multi-part identifier. Subclasses should override this method to + * provide proper multi-part identifier parsing with access to a full SQL parser. + * + * For example, in AstBuilder, this would parse "`catalog`.`schema`.`table`" into Seq("catalog", + * "schema", "table"). + * + * This method is only called when parsing IDENTIFIER('literal') where the literal contains a + * qualified identifier (e.g., IDENTIFIER('schema.table')). Since DataTypeAstBuilder only parses + * data types (not full SQL with qualified table names), this should never be called in + * practice. The base implementation throws an error to catch unexpected usage. + * + * @param identifier + * The identifier string to parse, potentially containing dots and backticks. + * @return + * Sequence of identifier parts. + */ + protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + throw SparkException.internalError( + "parseMultipartIdentifier must be overridden by subclasses. " + + s"Attempted to parse: $identifier") + } + + /** + * Get the identifier parts from a context, handling both regular identifiers and + * IDENTIFIER('literal'). This method is used to support identifier-lite syntax where + * IDENTIFIER('string') is folded at parse time. For qualified identifiers like + * IDENTIFIER('`catalog`.`schema`'), this will parse the string and return multiple parts. + * + * Subclasses should override this method to provide actual parsing logic. + */ + protected def getIdentifierParts(ctx: ParserRuleContext): Seq[String] = { + ctx match { + case idCtx: IdentifierContext => + // identifier can be either strictIdentifier or strictNonReserved. + // Recursively process the strictIdentifier. + Option(idCtx.strictIdentifier()).map(getIdentifierParts).getOrElse(Seq(ctx.getText)) + + case idLitCtx: IdentifierLiteralContext => + // For IDENTIFIER('literal') in strictIdentifier. + val literalValue = string(visitStringLit(idLitCtx.stringLit())) + // Parse the string to handle qualified identifiers like "`cat`.`schema`". + parseMultipartIdentifier(literalValue) + + case errCapture: ErrorCapturingIdentifierContext => + // Regular identifier with errorCapturingIdentifierExtra. + // Need to recursively handle identifier which might itself be IDENTIFIER('literal'). + Option(errCapture.identifier()) + .flatMap(id => Option(id.strictIdentifier()).map(getIdentifierParts)) + .getOrElse(Seq(ctx.getText)) + + case _ => + // For regular identifiers, just return the text as a single part. + Seq(ctx.getText) + } + } + + /** + * Get the text of a SINGLE identifier, handling both regular identifiers and + * IDENTIFIER('literal'). This method REQUIRES that the identifier be unqualified (single part + * only). If IDENTIFIER('qualified.name') is used where a single identifier is required, this + * will error. + */ + protected def getIdentifierText(ctx: ParserRuleContext): String = { + val parts = getIdentifierParts(ctx) + if (parts.size > 1) { + throw new ParseException( + errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + messageParameters = Map("identifier" -> toSQLId(parts), "limit" -> "1"), + ctx) + } + parts.head + } + + /** + * Create a multi-part identifier. Handles identifier-lite with qualified identifiers like + * IDENTIFIER('`cat`.`schema`').table */ override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - ctx.parts.asScala.map(_.getText).toSeq + // Each part is an errorCapturingIdentifier (which wraps identifier). + // getIdentifierParts recursively handles IDENTIFIER('literal') syntax through + // identifier -> strictIdentifier -> identifierLiteral. + ctx.parts.asScala.flatMap(getIdentifierParts).toSeq } /** @@ -351,7 +469,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { } StructField( - name = colName.getText, + name = getIdentifierText(colName), dataType = typedVisit[DataType](ctx.dataType), nullable = NULL == null, metadata = builder.build()) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala index 8beeb9b17d4c8..f32c1d6f3836d 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala @@ -81,7 +81,8 @@ class SubstituteParmsAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { */ override def visitNamedParameterLiteral(ctx: NamedParameterLiteralContext): AnyRef = withOrigin(ctx) { - val paramName = ctx.namedParameterMarker().identifier().getText + // Named parameters use simpleIdentifier, so .getText() is correct. + val paramName = ctx.namedParameterMarker().simpleIdentifier().getText namedParams += paramName // Calculate the location of the entire parameter (including the colon) @@ -117,7 +118,8 @@ class SubstituteParmsAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { */ override def visitNamedParameterMarkerRule(ctx: NamedParameterMarkerRuleContext): AnyRef = withOrigin(ctx) { - val paramName = ctx.namedParameterMarker().identifier().getText + // Named parameters use simpleIdentifier, so .getText() is correct. + val paramName = ctx.namedParameterMarker().simpleIdentifier().getText namedParams += paramName // Calculate the location of the entire parameter (including the colon) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index e2e320be36546..32270df0a9885 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -430,7 +430,15 @@ case class UnclosedCommentProcessor(command: String, tokenStream: CommonTokenStr } object DataTypeParser extends AbstractParser { - override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder + override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder { + // DataTypeParser only parses data types, not full SQL. + // Multi-part identifiers should not appear in IDENTIFIER() within type definitions. + override protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + throw SparkException.internalError( + "DataTypeParser does not support multi-part identifiers in IDENTIFIER(). " + + s"Attempted to parse: $identifier") + } + } } object AbstractParser extends Logging { @@ -476,6 +484,7 @@ object AbstractParser extends Logging { parser.SQL_standard_keyword_behavior = conf.enforceReservedKeywords parser.double_quoted_identifiers = conf.doubleQuotedIdentifiers parser.parameter_substitution_enabled = !conf.legacyParameterSubstitutionConstantsOnly + parser.legacy_identifier_clause_only = conf.legacyIdentifierClauseOnly } /** diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 630f274a621e9..553161ea2db0a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -477,7 +477,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { ctx) } - def showFunctionsUnsupportedError(identifier: String, ctx: IdentifierContext): Throwable = { + def showFunctionsUnsupportedError(identifier: String, ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_SCOPE", messageParameters = Map("scope" -> toSQLId(identifier)), diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index 0973750c65ce4..1be00d75acbce 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -51,6 +51,7 @@ private[sql] trait SqlApiConf { def parserDfaCacheFlushThreshold: Int def parserDfaCacheFlushRatio: Double def legacyParameterSubstitutionConstantsOnly: Boolean + def legacyIdentifierClauseOnly: Boolean } private[sql] object SqlApiConf { @@ -106,4 +107,5 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf { override def parserDfaCacheFlushThreshold: Int = -1 override def parserDfaCacheFlushRatio: Double = -1.0 override def legacyParameterSubstitutionConstantsOnly: Boolean = false + override def legacyIdentifierClauseOnly: Boolean = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 971633b9a46a8..d1d4a6b8c9800 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -118,6 +118,15 @@ class AstBuilder extends DataTypeAstBuilder } } + /** + * Override to provide actual multi-part identifier parsing using CatalystSqlParser. This allows + * the base class to handle IDENTIFIER('qualified.identifier') without needing special case + * logic in getIdentifierParts. + */ + override protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + CatalystSqlParser.parseMultipartIdentifier(identifier) + } + /** * Retrieves the original input text for a given parser context, preserving all whitespace and * formatting. @@ -255,12 +264,6 @@ class AstBuilder extends DataTypeAstBuilder private def visitDeclareConditionStatementImpl( ctx: DeclareConditionStatementContext): ErrorCondition = { - // Qualified user defined condition name is not allowed. - if (ctx.multipartIdentifier().parts.size() > 1) { - throw SqlScriptingErrors - .conditionCannotBeQualified(CurrentOrigin.get, ctx.multipartIdentifier().getText) - } - // If SQLSTATE is not provided, default to 45000. val sqlState = Option(ctx.sqlStateValue()) .map(sqlStateValueContext => string(visitStringLit(sqlStateValueContext.stringLit()))) @@ -269,7 +272,7 @@ class AstBuilder extends DataTypeAstBuilder assertSqlState(sqlState) // Get condition name. - val conditionName = visitMultipartIdentifier(ctx.multipartIdentifier()).head + val conditionName = getIdentifierText(ctx.strictIdentifier()) assertConditionName(conditionName) @@ -561,15 +564,15 @@ class AstBuilder extends DataTypeAstBuilder val query = withOrigin(queryCtx) { SingleStatement(visitQuery(queryCtx)) } - parsingCtx.labelContext.enterForScope(Option(ctx.multipartIdentifier())) - val varName = Option(ctx.multipartIdentifier()).map(_.getText) + parsingCtx.labelContext.enterForScope(Option(ctx.strictIdentifier())) + val varName = Option(ctx.strictIdentifier()).map(getIdentifierText) val body = visitCompoundBodyImpl( ctx.compoundBody(), None, parsingCtx, isScope = false ) - parsingCtx.labelContext.exitForScope(Option(ctx.multipartIdentifier())) + parsingCtx.labelContext.exitForScope(Option(ctx.strictIdentifier())) parsingCtx.labelContext.exitLabeledScope(Option(ctx.beginLabel())) ForStatement(query, varName, body, Some(labelText)) @@ -580,26 +583,26 @@ class AstBuilder extends DataTypeAstBuilder ctx match { case c: BeginEndCompoundBlockContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => if (isIterate) { throw SqlScriptingErrors.invalidIterateLabelUsageForCompound(CurrentOrigin.get, label) } true case c: WhileStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case c: RepeatStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case c: LoopStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case c: ForStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case _ => false } @@ -607,7 +610,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitLeaveStatement(ctx: LeaveStatementContext): LeaveStatement = withOrigin(ctx) { - val labelText = ctx.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + val labelText = getIdentifierText(ctx.strictIdentifier()).toLowerCase(Locale.ROOT) var parentCtx = ctx.parent while (Option(parentCtx).isDefined) { @@ -623,7 +626,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitIterateStatement(ctx: IterateStatementContext): IterateStatement = withOrigin(ctx) { - val labelText = ctx.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + val labelText = getIdentifierText(ctx.strictIdentifier()).toLowerCase(Locale.ROOT) var parentCtx = ctx.parent while (Option(parentCtx).isDefined) { @@ -797,7 +800,8 @@ class AstBuilder extends DataTypeAstBuilder (columnAliases, plan) => UnresolvedSubqueryColumnAliases(visitIdentifierList(columnAliases), plan) ) - SubqueryAlias(ctx.name.getText, subQuery) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + SubqueryAlias(getIdentifierText(ctx.name), subQuery) } /** @@ -1786,7 +1790,8 @@ class AstBuilder extends DataTypeAstBuilder // Collect all window specifications defined in the WINDOW clause. val baseWindowTuples = ctx.namedWindow.asScala.map { wCtx => - (wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec)) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + (getIdentifierText(wCtx.name), typedVisit[WindowSpec](wCtx.windowSpec)) } baseWindowTuples.groupBy(_._1).foreach { kv => if (kv._2.size > 1) { @@ -1927,6 +1932,7 @@ class AstBuilder extends DataTypeAstBuilder query: LogicalPlan): LogicalPlan = withOrigin(ctx) { var plan = query ctx.hintStatements.asScala.reverse.foreach { stmt => + // Hint names use simpleIdentifier, so .getText() is correct. plan = UnresolvedHint(stmt.hintName.getText, stmt.parameters.asScala.map(expression).toSeq, plan) } @@ -1974,17 +1980,18 @@ class AstBuilder extends DataTypeAstBuilder // this is needed to create unpivot and to filter unpivot for nulls further down val valueColumnNames = Option(ctx.unpivotOperator().unpivotSingleValueColumnClause()) - .map(_.unpivotValueColumn().identifier().getText) + .map(vc => getIdentifierText(vc.unpivotValueColumn().identifier())) .map(Seq(_)) .getOrElse( Option(ctx.unpivotOperator().unpivotMultiValueColumnClause()) - .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq) + .map(_.unpivotValueColumns.asScala.map(vc => + getIdentifierText(vc.identifier())).toSeq) .get ) val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) { val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause() - val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText + val variableColumnName = getIdentifierText(unpivotClause.unpivotNameColumn().identifier()) val (unpivotColumns, unpivotAliases) = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumnAndAlias).toSeq.unzip @@ -1999,7 +2006,7 @@ class AstBuilder extends DataTypeAstBuilder ) } else { val unpivotClause = ctx.unpivotOperator().unpivotMultiValueColumnClause() - val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText + val variableColumnName = getIdentifierText(unpivotClause.unpivotNameColumn().identifier()) val (unpivotColumns, unpivotAliases) = unpivotClause.unpivotColumnSets.asScala.map(visitUnpivotColumnSet).toSeq.unzip @@ -2043,7 +2050,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitUnpivotColumnAndAlias(ctx: UnpivotColumnAndAliasContext): (NamedExpression, Option[String]) = withOrigin(ctx) { val attr = visitUnpivotColumn(ctx.unpivotColumn()) - val alias = Option(ctx.unpivotAlias()).map(_.errorCapturingIdentifier().getText) + val alias = Option(ctx.unpivotAlias()).map(a => getIdentifierText(a.errorCapturingIdentifier())) (attr, alias) } @@ -2055,7 +2062,8 @@ class AstBuilder extends DataTypeAstBuilder (Seq[NamedExpression], Option[String]) = withOrigin(ctx) { val exprs = ctx.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq - val alias = Option(ctx.unpivotAlias()).map(_.errorCapturingIdentifier().getText) + val alias = + Option(ctx.unpivotAlias()).map(a => getIdentifierText(a.errorCapturingIdentifier())) (exprs, alias) } @@ -2071,9 +2079,9 @@ class AstBuilder extends DataTypeAstBuilder unrequiredChildIndex = Nil, outer = ctx.OUTER != null, // scalastyle:off caselocale - Some(ctx.tblName.getText.toLowerCase), + Some(getIdentifierText(ctx.tblName).toLowerCase), // scalastyle:on caselocale - ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.quoted).toSeq, + ctx.colName.asScala.map(getIdentifierText).map(UnresolvedAttribute.quoted).toSeq, query) } @@ -2514,7 +2522,8 @@ class AstBuilder extends DataTypeAstBuilder * Create an alias ([[SubqueryAlias]]) for a [[LogicalPlan]]. */ private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = { - SubqueryAlias(alias.getText, plan) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + SubqueryAlias(getIdentifierText(alias), plan) } /** @@ -2544,9 +2553,11 @@ class AstBuilder extends DataTypeAstBuilder /** * Create a Sequence of Strings for an identifier list. + * Each identifier must be unqualified. + * Handles both regular identifiers and IDENTIFIER('literal'). */ override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) { - ctx.ident.asScala.map(_.getText).toSeq + ctx.ident.asScala.map(id => getIdentifierText(id)).toSeq } /* ******************************************************************************************** @@ -2554,18 +2565,20 @@ class AstBuilder extends DataTypeAstBuilder * ******************************************************************************************** */ /** * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern. + * Handles identifier-lite with qualified identifiers. */ override def visitTableIdentifier( ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) { - TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText)) + TableIdentifier(getIdentifierText(ctx.table), Option(ctx.db).map(getIdentifierText)) } /** * Create a [[FunctionIdentifier]] from a 'functionName' or 'databaseName'.'functionName' pattern. + * Handles identifier-lite with qualified identifiers. */ override def visitFunctionIdentifier( ctx: FunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) { - FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText)) + FunctionIdentifier(getIdentifierText(ctx.function), Option(ctx.db).map(getIdentifierText)) } /* ******************************************************************************************** @@ -2639,7 +2652,8 @@ class AstBuilder extends DataTypeAstBuilder override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) { val e = expression(ctx.expression) if (ctx.name != null) { - Alias(e, ctx.name.getText)() + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + Alias(e, getIdentifierText(ctx.name))() } else if (ctx.identifierList != null) { MultiAlias(e, visitIdentifierList(ctx.identifierList)) } else { @@ -2972,7 +2986,8 @@ class AstBuilder extends DataTypeAstBuilder } } else { // If the parser is not in ansi mode, we should return `UnresolvedAttribute`, in case there - // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP` or `CURRENT_TIME` + // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP` or `CURRENT_TIME`. + // ctx.name is a token, not an identifier context. UnresolvedAttribute.quoted(ctx.name.getText) } } @@ -3217,7 +3232,8 @@ class AstBuilder extends DataTypeAstBuilder * Create a reference to a window frame, i.e. [[WindowSpecReference]]. */ override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) { - WindowSpecReference(ctx.name.getText) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + WindowSpecReference(getIdentifierText(ctx.name)) } /** @@ -3353,9 +3369,11 @@ class AstBuilder extends DataTypeAstBuilder * it can be [[UnresolvedExtractValue]]. */ override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) { - val attr = ctx.fieldName.getText + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + val attr = getIdentifierText(ctx.fieldName) expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => + // For regex check, we need the original text before identifier-lite resolution ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName && @@ -3393,13 +3411,17 @@ class AstBuilder extends DataTypeAstBuilder * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { + // For regex check, we need the original text before identifier-lite resolution ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName && isRegex(columnNameRegex) && canApplyRegex(ctx) => UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) case _ => - UnresolvedAttribute.quoted(ctx.getText) + // Use getIdentifierParts to handle IDENTIFIER('literal') correctly + // This allows IDENTIFIER('t').c1 to work like t.c1 + val parts = getIdentifierParts(ctx.identifier()) + UnresolvedAttribute(parts) } } @@ -4046,7 +4068,7 @@ class AstBuilder extends DataTypeAstBuilder ctx: ColDefinitionContext): ColumnAndConstraint = withOrigin(ctx) { import ctx._ - val name: String = colName.getText + val name: String = getIdentifierText(colName) // Check that no duplicates exist among any CREATE TABLE column options specified. var nullable = true var defaultExpression: Option[DefaultExpressionContext] = None @@ -4118,7 +4140,7 @@ class AstBuilder extends DataTypeAstBuilder ctx: ColumnConstraintDefinitionContext): TableConstraint = { withOrigin(ctx) { val name = if (ctx.name != null) { - ctx.name.getText + getIdentifierText(ctx.name) } else { null } @@ -5460,7 +5482,8 @@ class AstBuilder extends DataTypeAstBuilder invalidStatement("ALTER TABLE ... PARTITION ... CHANGE COLUMN", ctx) } val columnNameParts = typedVisit[Seq[String]](ctx.colName) - if (!conf.resolver(columnNameParts.last, ctx.colType().colName.getText)) { + if (!conf.resolver(columnNameParts.last, + getIdentifierText(ctx.colType().colName))) { throw QueryParsingErrors.operationInHiveStyleCommandUnsupportedError("Renaming column", "ALTER COLUMN", ctx, Some("please run RENAME COLUMN instead")) } @@ -5587,7 +5610,7 @@ class AstBuilder extends DataTypeAstBuilder ctx: TableConstraintDefinitionContext): TableConstraint = withOrigin(ctx) { val name = if (ctx.name != null) { - ctx.name.getText + getIdentifierText(ctx.name) } else { null } @@ -5691,7 +5714,7 @@ class AstBuilder extends DataTypeAstBuilder ctx.identifierReference, "ALTER TABLE ... DROP CONSTRAINT") DropConstraint( table, - ctx.name.getText, + getIdentifierText(ctx.name), ifExists = ctx.EXISTS() != null, cascade = ctx.CASCADE() != null) } @@ -5924,8 +5947,6 @@ class AstBuilder extends DataTypeAstBuilder * }}} */ override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val query = Option(ctx.query).map(plan) withIdentClause(ctx.identifierReference, query.toSeq, (ident, children) => { if (query.isDefined && ident.length > 1) { @@ -6303,12 +6324,14 @@ class AstBuilder extends DataTypeAstBuilder * Create a plan for a SHOW FUNCTIONS command. */ override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { - val (userScope, systemScope) = Option(ctx.identifier) - .map(_.getText.toLowerCase(Locale.ROOT)) match { + // Function scope uses simpleIdentifier, so .getText() is correct. + val scope = Option(ctx.functionScope) + val (userScope, systemScope) = scope.map(_.getText.toLowerCase(Locale.ROOT)) match { case None | Some("all") => (true, true) case Some("system") => (false, true) case Some("user") => (true, false) - case Some(x) => throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.identifier()) + case Some(x) => + throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.functionScope) } val legacy = Option(ctx.legacy).map(visitMultipartIdentifier) @@ -6435,11 +6458,13 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitTimestampadd(ctx: TimestampaddContext): Expression = withOrigin(ctx) { if (ctx.invalidUnit != null) { + // ctx.name and ctx.invalidUnit are tokens, not identifier contexts. throw QueryParsingErrors.invalidDatetimeUnitError( ctx, ctx.name.getText, ctx.invalidUnit.getText) } else { + // ctx.unit is a token, not an identifier context. TimestampAdd(ctx.unit.getText, expression(ctx.unitsAmount), expression(ctx.timestamp)) } } @@ -6449,11 +6474,13 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitTimestampdiff(ctx: TimestampdiffContext): Expression = withOrigin(ctx) { if (ctx.invalidUnit != null) { + // ctx.name and ctx.invalidUnit are tokens, not identifier contexts. throw QueryParsingErrors.invalidDatetimeUnitError( ctx, ctx.name.getText, ctx.invalidUnit.getText) } else { + // ctx.unit is a token, not an identifier context. TimestampDiff(ctx.unit.getText, expression(ctx.startTimestamp), expression(ctx.endTimestamp)) } } @@ -6463,7 +6490,8 @@ class AstBuilder extends DataTypeAstBuilder * */ override def visitNamedParameterLiteral( ctx: NamedParameterLiteralContext): Expression = withOrigin(ctx) { - NamedParameter(ctx.namedParameterMarker().identifier().getText) + // Named parameters use simpleIdentifier, so .getText() is correct. + NamedParameter(ctx.namedParameterMarker().simpleIdentifier().getText) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index a19b4cca28173..336db1382f898 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -28,7 +28,7 @@ import org.antlr.v4.runtime.tree.{ParseTree, TerminalNodeImpl} import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier -import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BeginLabelContext, EndLabelContext, MultipartIdentifierContext} +import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BeginLabelContext, EndLabelContext, StrictIdentifierContext} import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, ErrorCondition} import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.SparkParserUtils @@ -279,34 +279,49 @@ class SqlScriptingLabelContext { * @param beginLabelCtx Begin label context. * @param endLabelCtx The end label context. */ + /** + * Get label text from label context, handling IDENTIFIER() syntax. + */ + private def getLabelText(ctx: ParserRuleContext): String = { + val astBuilder = new DataTypeAstBuilder { + override protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + CatalystSqlParser.parseMultipartIdentifier(identifier) + } + } + val parts = astBuilder.extractIdentifierParts(ctx) + if (parts.size > 1) { + throw new ParseException( + errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + messageParameters = Map("identifier" -> parts.map(part => s"`$part`").mkString("."), + "limit" -> "1"), + ctx) + } + parts.head + } + private def checkLabels( beginLabelCtx: Option[BeginLabelContext], - endLabelCtx: Option[EndLabelContext]) : Unit = { + endLabelCtx: Option[EndLabelContext]): Unit = { + // Check label matching and other constraints. (beginLabelCtx, endLabelCtx) match { // Throw an error if labels do not match. - case (Some(bl: BeginLabelContext), Some(el: EndLabelContext)) - if bl.multipartIdentifier().getText.toLowerCase(Locale.ROOT) != - el.multipartIdentifier().getText.toLowerCase(Locale.ROOT) => - withOrigin(bl) { - throw SqlScriptingErrors.labelsMismatch( - CurrentOrigin.get, - bl.multipartIdentifier().getText, - el.multipartIdentifier().getText) - } - // Throw an error if label is qualified. - case (Some(bl: BeginLabelContext), _) - if bl.multipartIdentifier().parts.size() > 1 => - withOrigin(bl) { - throw SqlScriptingErrors.labelCannotBeQualified( - CurrentOrigin.get, - bl.multipartIdentifier().getText.toLowerCase(Locale.ROOT) - ) + case (Some(bl: BeginLabelContext), Some(el: EndLabelContext)) => + val beginLabel = getLabelText(bl.strictIdentifier()).toLowerCase(Locale.ROOT) + val endLabel = getLabelText(el.strictIdentifier()).toLowerCase(Locale.ROOT) + if (beginLabel != endLabel) { + withOrigin(bl) { + throw SqlScriptingErrors.labelsMismatch( + CurrentOrigin.get, + getLabelText(bl.strictIdentifier()), + getLabelText(el.strictIdentifier())) + } } // Throw an error if end label exists without begin label. case (None, Some(el: EndLabelContext)) => withOrigin(el) { throw SqlScriptingErrors.endLabelWithoutBeginLabel( - CurrentOrigin.get, el.multipartIdentifier().getText) + CurrentOrigin.get, + getLabelText(el.strictIdentifier())) } case _ => } @@ -314,7 +329,7 @@ class SqlScriptingLabelContext { /** Check if the label is defined. */ private def isLabelDefined(beginLabelCtx: Option[BeginLabelContext]): Boolean = { - beginLabelCtx.map(_.multipartIdentifier().getText).isDefined + beginLabelCtx.isDefined } /** @@ -322,13 +337,13 @@ class SqlScriptingLabelContext { * If the identifier is contained within seenLabels, raise an exception. */ private def assertIdentifierNotInSeenLabels( - identifierCtx: Option[MultipartIdentifierContext]): Unit = { + identifierCtx: Option[StrictIdentifierContext]): Unit = { identifierCtx.foreach { ctx => - val identifierName = ctx.getText - if (seenLabels.contains(identifierName.toLowerCase(Locale.ROOT))) { + val identifierName = getLabelText(ctx).toLowerCase(Locale.ROOT) + if (seenLabels.contains(identifierName)) { withOrigin(ctx) { throw SqlScriptingErrors - .duplicateLabels(CurrentOrigin.get, identifierName.toLowerCase(Locale.ROOT)) + .duplicateLabels(CurrentOrigin.get, identifierName) } } } @@ -348,7 +363,7 @@ class SqlScriptingLabelContext { // Get label text and add it to seenLabels. val labelText = if (isLabelDefined(beginLabelCtx)) { - val txt = beginLabelCtx.get.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + val txt = getLabelText(beginLabelCtx.get.strictIdentifier()).toLowerCase(Locale.ROOT) if (seenLabels.contains(txt)) { withOrigin(beginLabelCtx.get) { throw SqlScriptingErrors.duplicateLabels(CurrentOrigin.get, txt) @@ -374,18 +389,18 @@ class SqlScriptingLabelContext { */ def exitLabeledScope(beginLabelCtx: Option[BeginLabelContext]): Unit = { if (isLabelDefined(beginLabelCtx)) { - seenLabels.remove(beginLabelCtx.get.multipartIdentifier().getText.toLowerCase(Locale.ROOT)) + seenLabels.remove(getLabelText(beginLabelCtx.get.strictIdentifier()).toLowerCase(Locale.ROOT)) } } /** * Enter a for loop scope. - * If the for loop variable is defined, it will be asserted to not be inside seenLabels; + * If the for loop variable is defined, it will be asserted to not be inside seenLabels. * Then, if the for loop variable is defined, it will be added to seenLabels. */ - def enterForScope(identifierCtx: Option[MultipartIdentifierContext]): Unit = { + def enterForScope(identifierCtx: Option[StrictIdentifierContext]): Unit = { identifierCtx.foreach { ctx => - val identifierName = ctx.getText + val identifierName = getLabelText(ctx) assertIdentifierNotInSeenLabels(identifierCtx) seenLabels.add(identifierName.toLowerCase(Locale.ROOT)) @@ -403,9 +418,9 @@ class SqlScriptingLabelContext { * Exit a for loop scope. * If the for loop variable is defined, it will be removed from seenLabels. */ - def exitForScope(identifierCtx: Option[MultipartIdentifierContext]): Unit = { + def exitForScope(identifierCtx: Option[StrictIdentifierContext]): Unit = { identifierCtx.foreach { ctx => - val identifierName = ctx.getText + val identifierName = getLabelText(ctx) seenLabels.remove(identifierName.toLowerCase(Locale.ROOT)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala index 54c8c2ec089f9..9beead0e64875 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala @@ -186,6 +186,8 @@ class SubstituteParamsParser extends Logging { /** * Apply a list of substitutions to the SQL text. + * Inserts a space separator when a parameter is immediately preceded by a quote + * to avoid back-to-back quotes after substitution. */ private def applySubstitutions(sqlText: String, substitutions: List[Substitution]): String = { // Sort substitutions by start position in reverse order to avoid offset issues @@ -193,9 +195,18 @@ class SubstituteParamsParser extends Logging { var result = sqlText sortedSubstitutions.foreach { substitution => - result = result.substring(0, substitution.start) + - substitution.replacement + - result.substring(substitution.end) + val prefix = result.substring(0, substitution.start) + val replacement = substitution.replacement + val suffix = result.substring(substitution.end) + + // Check if replacement is immediately preceded by a quote and doesn't already + // start with whitespace + val needsSpace = substitution.start > 0 && + (result(substitution.start - 1) == '\'' || result(substitution.start - 1) == '"') && + replacement.nonEmpty && !replacement(0).isWhitespace + + val space = if (needsSpace) " " else "" + result = s"$prefix$space$replacement$suffix" } result } @@ -211,4 +222,3 @@ object SubstituteParamsParser { positionalParams: List[String] = List.empty): (String, Int, PositionMapper) = instance.substitute(sqlText, namedParams, positionalParams) } - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index d00d639eb5de3..9381f2f6f2e34 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2293,13 +2293,17 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def identifierTooManyNamePartsError(originalIdentifier: String): Throwable = { new AnalysisException( errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", - messageParameters = Map("identifier" -> toSQLId(originalIdentifier))) + messageParameters = Map( + "identifier" -> toSQLId(originalIdentifier), + "limit" -> "2")) } def identifierTooManyNamePartsError(names: Seq[String]): Throwable = { new AnalysisException( errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", - messageParameters = Map("identifier" -> toSQLId(names))) + messageParameters = Map( + "identifier" -> toSQLId(names), + "limit" -> "2")) } def emptyMultipartIdentifierError(): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 1f7d2a149a7be..27aba1f7f2dfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -1120,10 +1120,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE cause = e) } - def ddlUnsupportedTemporarilyError(ddl: String): SparkUnsupportedOperationException = { + def ddlUnsupportedTemporarilyError( + ddl: String, + tableName: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2096", - messageParameters = Map("ddl" -> ddl)) + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + messageParameters = Map("tableName" -> toSQLId(tableName), "operation" -> ddl)) } def executeBroadcastTimeoutError(timeout: Long, ex: Option[TimeoutException]): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f3c6751a6eab7..3ec8366f8141e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -5004,6 +5004,17 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_IDENTIFIER_CLAUSE_ONLY = + buildConf("spark.sql.legacy.identifierClause") + .internal() + .doc("When set to false, IDENTIFIER('literal') is resolved to an identifier at parse time " + + "anywhere identifiers can occur. When set to true, only the legacy " + + " IDENTIFIER(constantExpr) clause is allowed, which evaluates the expression at analysis " + + " and is limited to a narrow subset of scenarios.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + val LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED = buildConf("spark.sql.legacy.allowNegativeScaleOfDecimal") .internal() @@ -7696,6 +7707,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { override def legacyParameterSubstitutionConstantsOnly: Boolean = getConf(SQLConf.LEGACY_PARAMETER_SUBSTITUTION_CONSTANTS_ONLY) + override def legacyIdentifierClauseOnly: Boolean = + getConf(SQLConf.LEGACY_IDENTIFIER_CLAUSE_ONLY) + def streamStatePollingInterval: Long = getConf(SQLConf.PIPELINES_STREAM_STATE_POLLING_INTERVAL) def watchdogMinRetryTimeInSeconds: Long = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index de4f8c4aa2cd7..30ecc902ff89f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -2275,11 +2275,11 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END; |END""".stripMargin checkError( - exception = intercept[SqlScriptingException] { + exception = intercept[ParseException] { parsePlan(sqlScriptText) }, - condition = "INVALID_LABEL_USAGE.QUALIFIED_LABEL_NAME", - parameters = Map("labelName" -> "PART1.PART2")) + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'.'", "hint" -> "")) } test("qualified label name: label cannot be qualified + end label") { @@ -2290,11 +2290,11 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END part1.part2; |END""".stripMargin checkError( - exception = intercept[SqlScriptingException] { + exception = intercept[ParseException] { parsePlan(sqlScriptText) }, - condition = "INVALID_LABEL_USAGE.QUALIFIED_LABEL_NAME", - parameters = Map("labelName" -> "PART1.PART2")) + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'.'", "hint" -> "")) } test("unique label names: nested labeled scope statements") { @@ -2787,13 +2787,13 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |BEGIN | DECLARE TEST.CONDITION CONDITION FOR SQLSTATE '12345'; |END""".stripMargin - val exception = intercept[SqlScriptingException] { + val exception = intercept[ParseException] { parsePlan(sqlScriptText) } checkError( exception = exception, - condition = "INVALID_ERROR_CONDITION_DECLARATION.QUALIFIED_CONDITION_NAME", - parameters = Map("conditionName" -> "TEST.CONDITION")) + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'FOR'", "hint" -> ": missing ';'")) assert(exception.origin.line.contains(3)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index f7876d9a023bd..c47e845416213 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -45,11 +45,11 @@ import org.apache.spark.sql.catalyst.analysis.{GeneralParameterizedQuery, NamePa import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, Literal} import org.apache.spark.sql.catalyst.parser.{HybridParameterContext, NamedParameterContext, ParserInterface, PositionalParameterContext} -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, OneRowRelation, Project, Range} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan, OneRowRelation, Project, Range} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.classic.SparkSession.applyAndLoadExtensions -import org.apache.spark.sql.errors.SqlScriptingErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, SqlScriptingErrors} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ExternalCommandExecutor import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -501,26 +501,31 @@ class SparkSession private( private[sql] def sql(sqlText: String, args: Array[_], tracker: QueryPlanningTracker): DataFrame = withActive { val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { - val parsedPlan = if (args.nonEmpty) { - // Resolve and validate parameters first - val paramMap = args.zipWithIndex.map { case (arg, idx) => - s"_pos_$idx" -> lit(arg).expr - }.toMap - val resolvedParams = resolveAndValidateParameters(paramMap) + val parsedPlan = { + // Always parse with parameter context to detect unbound parameter markers. + // Even if args is empty, we need to detect and reject parameter markers in the SQL. + val (paramMap, resolvedParams) = if (args.nonEmpty) { + val pMap = args.zipWithIndex.map { case (arg, idx) => + s"_pos_$idx" -> lit(arg).expr + }.toMap + (pMap, resolveAndValidateParameters(pMap)) + } else { + (Map.empty[String, Expression], Map.empty[String, Expression]) + } + val paramContext = PositionalParameterContext(resolvedParams.values.toSeq) val parsed = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) + // Check for SQL scripting with positional parameters - if (parsed.isInstanceOf[CompoundBody]) { + if (parsed.isInstanceOf[CompoundBody] && args.nonEmpty) { throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() } // In legacy mode, wrap with PosParameterizedQuery for analyzer binding - if (sessionState.conf.legacyParameterSubstitutionConstantsOnly) { + if (args.nonEmpty && sessionState.conf.legacyParameterSubstitutionConstantsOnly) { PosParameterizedQuery(parsed, paramContext.params) } else { parsed } - } else { - sessionState.sqlParser.parsePlan(sqlText) } parsedPlan } @@ -554,30 +559,29 @@ class SparkSession private( args: Map[String, Any], tracker: QueryPlanningTracker): DataFrame = withActive { - // Always set parameter context if we have actual parameters - if (args.nonEmpty) { - // Resolve and validate parameters first - val resolvedParams = resolveAndValidateParameters(args.transform((_, v) => lit(v).expr)) - val paramContext = NamedParameterContext(resolvedParams) - val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { - val parsedPlan = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) - // In legacy mode, wrap the parsed plan with NameParameterizedQuery - // so that the BindParameters analyzer rule can bind the parameters - if (sessionState.conf.legacyParameterSubstitutionConstantsOnly) { - NameParameterizedQuery(parsedPlan, paramContext.params) - } else { - parsedPlan - } - } - - Dataset.ofRows(self, plan, tracker) + // Always parse with parameter context to detect unbound parameter markers. + // Even if args is empty, we need to detect and reject parameter markers in the SQL. + val resolvedParams = if (args.nonEmpty) { + resolveAndValidateParameters(args.transform((_, v) => lit(v).expr)) } else { - // No parameters - parse normally without parameter context - val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { - sessionState.sqlParser.parsePlan(sqlText) + Map.empty[String, Expression] + } + val paramContext = NamedParameterContext(resolvedParams) + val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { + val parsedPlan = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) + val queryPlan = parsedPlan match { + case compoundBody: CompoundBody => compoundBody + case logicalPlan: LogicalPlan => + // In legacy mode, wrap with NameParameterizedQuery for analyzer binding + if (args.nonEmpty && sessionState.conf.legacyParameterSubstitutionConstantsOnly) { + NameParameterizedQuery(logicalPlan, paramContext.params) + } else { + logicalPlan + } } - Dataset.ofRows(self, plan, tracker) + queryPlan } + Dataset.ofRows(self, plan, tracker) } /** @inheritdoc */ @@ -610,6 +614,8 @@ class SparkSession private( tracker: QueryPlanningTracker): DataFrame = withActive { val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { + // Always parse with parameter context to detect unbound parameter markers. + // Even if args is empty, we need to detect and reject parameter markers in the SQL. val parsedPlan = if (args.nonEmpty) { // Resolve and validate parameter arguments val paramMap = args.zipWithIndex.map { case (arg, idx) => @@ -643,11 +649,6 @@ class SparkSession private( val parsed = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) - // Check for SQL scripting with positional parameters - if (parsed.isInstanceOf[CompoundBody] && paramNames.isEmpty) { - throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() - } - // In legacy mode, wrap with GeneralParameterizedQuery for analyzer binding if (sessionState.conf.legacyParameterSubstitutionConstantsOnly) { GeneralParameterizedQuery( @@ -659,8 +660,16 @@ class SparkSession private( parsed } } else { - sessionState.sqlParser.parsePlan(sqlText) + // No arguments provided, but still need to detect parameter markers + val paramContext = HybridParameterContext(Seq.empty, Seq.empty) + sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) } + + // Check for SQL scripts in EXECUTE IMMEDIATE (applies to both empty and non-empty args) + if (parsedPlan.isInstanceOf[CompoundBody]) { + throw QueryCompilationErrors.sqlScriptInExecuteImmediate(sqlText) + } + parsedPlan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 58bffbed3e69e..550c23e3e830c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -118,14 +118,22 @@ class SparkSqlParser extends AbstractSqlParser { // Step 2: Apply parameter substitution if a parameter context is provided. val (paramSubstituted, positionMapper, hasParameters) = parameterContext match { case Some(context) => + // Check if the context actually contains parameters + val contextHasParams = context match { + case NamedParameterContext(params) => params.nonEmpty + case PositionalParameterContext(params) => params.nonEmpty + case HybridParameterContext(args, _) => args.nonEmpty + } if (SQLConf.get.legacyParameterSubstitutionConstantsOnly) { // Legacy mode: Parameters are detected but substitution is deferred to analysis phase. - (variableSubstituted, PositionMapper.identity(variableSubstituted), true) + // Only set hasParameters if the context actually contains parameters. + (variableSubstituted, PositionMapper.identity(variableSubstituted), contextHasParams) } else { // Modern mode: Perform parameter substitution during parsing. val (substituted, mapper) = ParameterHandler.substituteParameters(variableSubstituted, context) - (substituted, mapper, true) + // Only set hasParameters if the context actually contains parameters. + (substituted, mapper, contextHasParams) } case None => // No parameter context provided; skip parameter substitution. @@ -629,7 +637,8 @@ class SparkSqlAstBuilder extends AstBuilder { val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl => icl.identifierComment.asScala.map { ic => - ic.identifier.getText -> Option(ic.commentSpec()).map(visitCommentSpec) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + getIdentifierText(ic.identifier) -> Option(ic.commentSpec()).map(visitCommentSpec) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 3c36d3e2d4173..5efad83bcba78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -23,6 +23,7 @@ import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{execution, AnalysisException} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NamedRelation} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper, NormalizeFloatingNumbers} @@ -31,12 +32,14 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.{SparkStrategy => Strategy} import org.apache.spark.sql.execution.aggregate.AggUtils import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{WriteFiles, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.{LogicalRelation, WriteFiles, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.exchange.{REBALANCE_PARTITIONS_BY_COL, REBALANCE_PARTITIONS_BY_NONE, REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeExec} import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.execution.python.streaming.{FlatMapGroupsInPandasWithStateExec, TransformWithStateInPySparkExec} @@ -1091,10 +1094,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering, r.stream) :: Nil - case _: UpdateTable => - throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("UPDATE TABLE") - case _: MergeIntoTable => - throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("MERGE INTO TABLE") + case u: UpdateTable => + val tableName = extractTableNameForError(u.table) + throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("UPDATE TABLE", tableName) + case m: MergeIntoTable => + val tableName = extractTableNameForError(m.targetTable) + throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("MERGE INTO TABLE", tableName) case logical.CollectMetrics(name, metrics, child, _) => execution.CollectMetricsExec(name, metrics, planLater(child)) :: Nil case WriteFiles(child, fileFormat, partitionColumns, bucket, options, staticPartitions) => @@ -1105,4 +1110,22 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => Nil } } + + /** + * Extracts a user-friendly table name from a logical plan for error messages. + */ + private def extractTableNameForError(table: LogicalPlan): String = { + val unwrapped = EliminateSubqueryAliases(table) + unwrapped match { + // Check specific types before NamedRelation since they extend it + case DataSourceV2Relation(_, _, catalog, Some(ident), _, _) => + (catalog.map(_.name()).toSeq ++ ident.asMultipartIdentifier).mkString(".") + case LogicalRelation(_, _, Some(catalogTable), _, _) => + catalogTable.identifier.unquotedString + case r: NamedRelation => + r.name + case _ => + "unknown" + } + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index 1271f730d1e53..c874945badb1b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -1224,3 +1224,43 @@ EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP(1, 'one', 2, -- !query analysis Project [typeof(map(1, one, 2, two)) AS type#x, map(1, one, 2, two) AS val#x] +- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE 'SELECT :param' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "param" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : ":param" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT ?' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "_7" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "?" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out new file mode 100644 index 0000000000000..17fcc9b47729e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -0,0 +1,2431 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET hivevar:colname = 'c' +-- !query analysis +SetCommand (hivevar:colname,Some('c')) + + +-- !query +SELECT IDENTIFIER(${colname} || '_1') FROM VALUES(1) AS T(c_1) +-- !query analysis +Project [c_1#x] ++- SubqueryAlias T + +- LocalRelation [c_1#x] + + +-- !query +SELECT IDENTIFIER('c1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT IDENTIFIER('t.c1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT IDENTIFIER('`t`.c1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT IDENTIFIER('`c 1`') FROM VALUES(1) AS T(`c 1`) +-- !query analysis +Project [c 1#x] ++- SubqueryAlias T + +- LocalRelation [c 1#x] + + +-- !query +SELECT IDENTIFIER('``') FROM VALUES(1) AS T(``) +-- !query analysis +Project [#x] ++- SubqueryAlias T + +- LocalRelation [#x] + + +-- !query +SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +CREATE SCHEMA IF NOT EXISTS s +-- !query analysis +CreateNamespace true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [s] + + +-- !query +CREATE TABLE s.tab(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`s`.`tab`, false + + +-- !query +USE SCHEMA s +-- !query analysis +SetNamespaceCommand [s] + + +-- !query +INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/s.db/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/s.db/tab], Append, `spark_catalog`.`s`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/s.db/tab), [c1] ++- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +DELETE FROM IDENTIFIER('ta' || 'b') WHERE 1=0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "DELETE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 +-- !query analysis +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s.c1 = t.c1 + WHEN MATCHED THEN UPDATE SET c1 = 3 +-- !query analysis +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('tab') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +SELECT * FROM IDENTIFIER('s.tab') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +SELECT * FROM IDENTIFIER('`s`.`tab`') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +SELECT * FROM IDENTIFIER('t' || 'a' || 'b') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +USE SCHEMA default +-- !query analysis +SetNamespaceCommand [default] + + +-- !query +DROP TABLE s.tab +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), s.tab + + +-- !query +DROP SCHEMA s +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [s] + + +-- !query +SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1) +-- !query analysis +Project [coalesce(cast(null as int), 1) AS coalesce(NULL, 1)#x] ++- OneRowRelation + + +-- !query +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query analysis +Project [abs(c1#x) AS abs(c1)#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1) +-- !query analysis +Project [id#xL] ++- Range (0, 1, step=1) + + +-- !query +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE SCHEMA identifier_clauses +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clauses] + + +-- !query +USE identifier_clauses +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clauses] + + +-- !query +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab') +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clauses.tab + + +-- !query +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false + + +-- !query +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "REPLACE TABLE", + "tableName" : "`spark_catalog`.`identifier_clauses`.`tab`" + } +} + + +-- !query +CACHE TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +CacheTable [tab], false, true + +- SubqueryAlias spark_catalog.identifier_clauses.tab + +- Relation spark_catalog.identifier_clauses.tab[c1#x] csv + + +-- !query +UNCACHE TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +UncacheTable false, true + +- SubqueryAlias spark_catalog.identifier_clauses.tab + +- Relation spark_catalog.identifier_clauses.tab[c1#x] csv + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clauses.tab + + +-- !query +USE default +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [default] + + +-- !query +DROP SCHEMA identifier_clauses +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clauses] + + +-- !query +CREATE TABLE tab(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +INSERT INTO tab VALUES (1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1] ++- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT c1 FROM tab +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.tab + +- Relation spark_catalog.default.tab[c1#x] csv + + +-- !query +DESCRIBE IDENTIFIER('ta' || 'b') +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`tab`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ANALYZE TABLE IDENTIFIER('ta' || 'b') COMPUTE STATISTICS +-- !query analysis +AnalyzeTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +ALTER TABLE IDENTIFIER('ta' || 'b') ADD COLUMN c2 INT +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`default`.`tab`, [StructField(c2,IntegerType,true)] + + +-- !query +SHOW TBLPROPERTIES IDENTIFIER('ta' || 'b') +-- !query analysis +ShowTableProperties [key#x, value#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tab, V1Table(default.tab), [c1#x, c2#x] + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('ta' || 'b') +-- !query analysis +ShowColumnsCommand `spark_catalog`.`default`.`tab`, [col_name#x] + + +-- !query +COMMENT ON TABLE IDENTIFIER('ta' || 'b') IS 'hello' +-- !query analysis +CommentOnTable hello ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tab, V1Table(default.tab), [c1#x, c2#x] + + +-- !query +REFRESH TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +RefreshTableCommand `spark_catalog`.`default`.`tab` + + +-- !query +REPAIR TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_PARTITIONED_TABLE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "MSCK REPAIR TABLE", + "tableIdentWithDB" : "`spark_catalog`.`default`.`tab`" + } +} + + +-- !query +TRUNCATE TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`tab` + + +-- !query +DROP TABLE IF EXISTS tab +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE OR REPLACE VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, [(c1,None)], VALUES(1), false, true, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM v +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(col1#x as int) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +ALTER VIEW IDENTIFIER('v') AS VALUES(2) +-- !query analysis +AlterViewAsCommand `spark_catalog`.`default`.`v`, VALUES(2), true + +- LocalRelation [col1#x] + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, false, true, false + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query analysis +CreateViewCommand `v`, [(c1,None)], VALUES(1), false, false, LocalTempView, UNSUPPORTED, true + +- LocalRelation [col1#x] + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query analysis +DropTempViewCommand v + + +-- !query +CREATE SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET PROPERTIES (somekey = 'somevalue') +-- !query analysis +SetNamespaceProperties [somekey=somevalue] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET LOCATION 'someloc' +-- !query analysis +SetNamespaceLocation someloc ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +COMMENT ON SCHEMA IDENTIFIER('id' || 'ent') IS 'some comment' +-- !query analysis +CommentOnNamespace some comment ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +DESCRIBE SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +DescribeNamespace false, [info_name#x, info_value#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +SHOW TABLES IN IDENTIFIER('id' || 'ent') +-- !query analysis +ShowTables [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +SHOW TABLE EXTENDED IN IDENTIFIER('id' || 'ent') LIKE 'hello' +-- !query analysis +ShowTablesCommand ident, hello, [namespace#x, tableName#x, isTemporary#x, information#x], true + + +-- !query +USE IDENTIFIER('id' || 'ent') +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +SHOW CURRENT SCHEMA +-- !query analysis +ShowCurrentNamespaceCommand + + +-- !query +USE SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +SetNamespaceCommand [ident] + + +-- !query +USE SCHEMA default +-- !query analysis +SetNamespaceCommand [default] + + +-- !query +DROP SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +CREATE SCHEMA ident +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +CREATE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +CreateFunctionCommand spark_catalog.ident.myDoubleAvg, test.org.apache.spark.sql.MyDoubleAvg, false, false, false + + +-- !query +DESCRIBE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query analysis +DescribeFunctionCommand org.apache.spark.sql.catalyst.expressions.ExpressionInfo@xxxxxxxx, false + + +-- !query +REFRESH FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query analysis +RefreshFunctionCommand ident, mydoubleavg + + +-- !query +DROP FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query analysis +DropFunctionCommand spark_catalog.ident.mydoubleavg, false, false + + +-- !query +DROP SCHEMA ident +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +CreateFunctionCommand myDoubleAvg, test.org.apache.spark.sql.MyDoubleAvg, true, false, false + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') +-- !query analysis +DropFunctionCommand myDoubleAvg, false, true + + +-- !query +DECLARE var = 'sometable' +-- !query analysis +CreateVariable defaultvalueexpression(sometable, 'sometable'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var + + +-- !query +CREATE TABLE IDENTIFIER(var)(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`sometable`, false + + +-- !query +SET VAR var = 'c1' +-- !query analysis +SetVariable [variablereference(system.session.var='sometable')] ++- Project [c1 AS var#x] + +- OneRowRelation + + +-- !query +SELECT IDENTIFIER(var) FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SET VAR var = 'some' +-- !query analysis +SetVariable [variablereference(system.session.var='c1')] ++- Project [some AS var#x] + +- OneRowRelation + + +-- !query +DROP TABLE IDENTIFIER(var || 'table') +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.sometable + + +-- !query +SELECT IDENTIFIER('c 1') FROM VALUES(1) AS T(`c 1`) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : ": extra input '1'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "IDENTIFIER('c 1')" + } ] +} + + +-- !query +SELECT IDENTIFIER('') FROM VALUES(1) AS T(``) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_EMPTY_STATEMENT", + "sqlState" : "42617", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "IDENTIFIER('')" + } ] +} + + +-- !query +VALUES(IDENTIFIER(CAST(NULL AS STRING))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NULL", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "CAST(NULL AS STRING)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 38, + "fragment" : "CAST(NULL AS STRING)" + } ] +} + + +-- !query +VALUES(IDENTIFIER(1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "substr('HELLO', 1, CAST((rand() + CAST(1 AS DOUBLE)) AS INT))", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 48, + "fragment" : "SUBSTR('HELLO', 1, RAND() + 1)" + } ] +} + + +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 25, + "fragment" : "1" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP TABLE IDENTIFIER('a.b.c') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP VIEW IDENTIFIER('a.b.c') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +COMMENT ON TABLE IDENTIFIER('a.b.c.d') IS 'hello' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +VALUES(IDENTIFIER(1)()) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER('a.b.c.d')()) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "IDENTIFIER('a.b.c.d')()" + } ] +} + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + "sqlState" : "42000", + "messageParameters" : { + "database" : "`default`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 108, + "fragment" : "CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + "sqlState" : "42000", + "messageParameters" : { + "name" : "`default`.`myDoubleAvg`", + "statement" : "DROP TEMPORARY FUNCTION" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + "sqlState" : "428EK", + "messageParameters" : { + "actualName" : "`default`.`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1)" + } ] +} + + +-- !query +create temporary view identifier('v1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query analysis +CreateViewCommand `v1`, (select my_col from (values (1), (2), (1) as (my_col)) group by 1), false, false, LocalTempView, UNSUPPORTED, true + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +cache table identifier('t1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query analysis +CacheTableAsSelect t1, (select my_col from (values (1), (2), (1) as (my_col)) group by 1), false, true + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +create table identifier('t2') using csv as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t2`, ErrorIfExists, [my_col] + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +insert into identifier('t2') select my_col from (values (3) as (my_col)) group by 1 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [my_col] ++- Project [my_col#x AS my_col#x] + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +drop view v1 +-- !query analysis +DropTempViewCommand v1 + + +-- !query +drop table t1 +-- !query analysis +DropTempViewCommand t1 + + +-- !query +drop table t2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2 + + +-- !query +DECLARE agg = 'max' +-- !query analysis +CreateVariable defaultvalueexpression(max, 'max'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.agg + + +-- !query +DECLARE col = 'c1' +-- !query analysis +CreateVariable defaultvalueexpression(c1, 'c1'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.col + + +-- !query +DECLARE tab = 'T' +-- !query analysis +CreateVariable defaultvalueexpression(T, 'T'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.tab + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias S +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias T +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias T + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias S +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias T +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias T + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias ABC +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias ABC + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + + +-- !query +SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''x.win''", + "hint" : "" + } +} + + +-- !query +SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "IDENTIFIER('t')" + } ] +} + + +-- !query +SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT * FROM s.IDENTIFIER('tab') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').tab +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''win''", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'AS'" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''my_table''", + "hint" : "" + } +} + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''v''", + "hint" : "" + } +} + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c1 FROM v +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 16, + "fragment" : "v" + } ] +} + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT c1 FROM tab +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT col1 FROM tab +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 20, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c2 FROM tab +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM tab_renamed +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab_renamed`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "tab_renamed" + } ] +} + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab_renamed + + +-- !query +DROP TABLE IF EXISTS tab +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_col_with_dot + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''schema.table''", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1.col2') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE SCHEMA identifier_clause_test_schema +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +USE identifier_clause_test_schema +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_show`, false + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query analysis +ShowViewsCommand identifier_clause_test_schema, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" + } +} + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show, V1Table(identifier_clause_test_schema.test_show), [c1#x, c2#x] + + +-- !query +DROP TABLE test_show +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE test_desc +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_desc + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_comment`, false + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query analysis +CommentOnTable table comment ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment, V1Table(identifier_clause_test_schema.test_comment), [c1#x, c2#x] + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE test_comment +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query analysis +AnalyzeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +RefreshTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table` + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +ShowColumnsCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, [col_name#x] + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_table + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query analysis +CreateVariable defaultvalueexpression(value, 'value'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing EQ" + } +} + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query analysis +Project [variablereference(system.session.my_var='value') AS variablereference(system.session.my_var='value')#x] ++- OneRowRelation + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query analysis +DropVariable false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT test_udf(5, 'hello') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "test_udf(5, 'hello')" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`" + } +} + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "sqlState" : "42883", + "messageParameters" : { + "name" : "`test_table_udf`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 32, + "fragment" : "test_table_udf(42)" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_table_udf`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query analysis +Project [c1#x.c2 AS c1.c2#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, false + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query analysis +Project [c1#x, c2#x] ++- Filter (c1#x = 1) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, false + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2), [c1, c3] ++- Project [col1#x AS c1#x, col2#x AS c3#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 103, + "fragment" : "SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query analysis +Project [c1#x, c2#x, rn#x] ++- Project [c1#x, c2#x, rn#x, rn#x] + +- Window [row_number() windowspecdefinition(c2#x, c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#x], [c2#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query analysis +Sort [c2#x ASC NULLS FIRST, count(c1)#xL ASC NULLS FIRST], true ++- Aggregate [c2#x], [c2#x, count(c1#x) AS count(c1)#xL] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query analysis +Sort [c1#x DESC NULLS LAST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 145, + "fragment" : "SELECT IDENTIFIER(concat(:schema, '.', :table, '.c1')) FROM VALUES(named_struct('c1', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema '.' :table))" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`test_view`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 46, + "fragment" : "IDENTIFIER(:view)" + } ] +} + + +-- !query +DROP VIEW test_view +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`identifier_clause_test_schema`.`test_view`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT map(:key, :val).IDENTIFIER(:key) AS result" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT IDENTIFIER(:alias '.c1') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query analysis +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "SELECT 1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test + + +-- !query +DROP TABLE integration_test2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test2 + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, false + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test), [arr] ++- Project [col1#x AS arr#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE lateral_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.lateral_test + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, false + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test), [id, a, b, c] ++- Project [col1#x AS id#x, col2#x AS a#x, col3#x AS b#x, col4#x AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE unpivot_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, false + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.hint_test + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 13d911c988381..fc8d225e22714 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -107,9 +107,11 @@ UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 -- !query analysis org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "UPDATE TABLE" + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -120,9 +122,11 @@ MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s. -- !query analysis org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "MERGE INTO TABLE" + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -853,7 +857,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", @@ -1064,27 +1069,32 @@ SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "''x.win''", - "hint" : "" - } + "identifier" : "`x`.`win`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 44, + "fragment" : "IDENTIFIER('x.win')" + } ] } -- !query SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +Project [c1#x] ++- Project [c1#x] + +- Join Inner, (c1#x = c1#x) + :- SubqueryAlias T1 + : +- LocalRelation [c1#x] + +- SubqueryAlias T2 + +- LocalRelation [c1#x] -- !query @@ -1111,40 +1121,28 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +Project [map(a, 1)[a] AS map(a, 1)[a]#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] -- !query SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +Project [named_struct(a, 1).a AS named_struct(a, 1).a#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] -- !query SELECT * FROM s.IDENTIFIER('tab') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", - "sqlState" : "42000", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "funcName" : "`s`.`IDENTIFIER`" + "relationName" : "`s`.`tab`" }, "queryContext" : [ { "objectType" : "", @@ -1159,110 +1157,1021 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "IDENTIFIER('s').IDENTIFIER('tab')" + } ] } -- !query SELECT * FROM IDENTIFIER('s').tab -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "IDENTIFIER('s').tab" + } ] } -- !query SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +Project [row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [c1#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [row_number() windowspecdefinition(c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x] + +- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query analysis +Project [row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [c1#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [row_number() windowspecdefinition(c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x] + +- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query analysis +Project [1 AS col1#x] ++- OneRowRelation + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias my_table + +- LocalRelation [c1#x, c2#x] + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias v +: +- Project [col1#x AS c1#x] +: +- LocalRelation [col1#x] ++- Project [c1#x] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, [(c1,None)], VALUES(1), false, true, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x] + + +-- !query +SELECT c1 FROM v +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(col1#x as int) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1] ++- Project [c1#x AS c1#x] + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT c1 FROM tab +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.tab + +- Relation spark_catalog.default.tab[c1#x] csv + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query analysis +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''win''", - "hint" : "" + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +SELECT col1 FROM tab -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "error" : "'WINDOW'", - "hint" : "" - } + "objectName" : "`col1`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "col1" + } ] } -- !query -WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +AlterTableAddColumnsCommand `spark_catalog`.`default`.`tab`, [StructField(c2,IntegerType,true)] + + +-- !query +SELECT c2 FROM tab +-- !query analysis +Project [c2#x] ++- SubqueryAlias spark_catalog.default.tab + +- Relation spark_catalog.default.tab[c1#x,c2#x] csv + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query analysis +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''v''", - "hint" : "" + "operation" : "DROP COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`default`.`tab`, `tab_renamed`, false + + +-- !query +SELECT * FROM tab_renamed +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias spark_catalog.default.tab_renamed + +- Relation spark_catalog.default.tab_renamed[c1#x,c2#x] csv + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab_renamed + + +-- !query +DROP TABLE IF EXISTS tab +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_col_with_dot`, false + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_col_with_dot + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : ": missing ')'" - } + "identifier" : "`schema`.`table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 65, + "fragment" : "VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2)" + } ] } -- !query -CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +SELECT 1 AS IDENTIFIER('col1.col2') -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : "" - } + "identifier" : "`col1`.`col2`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "1 AS IDENTIFIER('col1.col2')" + } ] } -- !query -CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +CREATE SCHEMA identifier_clause_test_schema -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +USE identifier_clause_test_schema +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_show`, false + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query analysis +ShowViewsCommand identifier_clause_test_schema, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query analysis +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : "" + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" } } + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show, V1Table(identifier_clause_test_schema.test_show), [c1#x, c2#x] + + +-- !query +DROP TABLE test_show +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE test_desc +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_desc + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_comment`, false + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query analysis +CommentOnTable table comment ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment, V1Table(identifier_clause_test_schema.test_comment), [c1#x, c2#x] + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`identifier_clause_test_schema`.`test_comment`, c1, StructField(c1,IntegerType,true) + + +-- !query +DROP TABLE test_comment +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query analysis +AnalyzeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +RefreshTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table` + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +ShowColumnsCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, [col_name#x] + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_table + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query analysis +CreateVariable defaultvalueexpression(value, 'value'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query analysis +SetVariable [variablereference(system.session.my_var='value')] ++- Project [new_value AS my_var#x] + +- OneRowRelation + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query analysis +Project [variablereference(system.session.my_var='new_value') AS variablereference(system.session.my_var='new_value')#x] ++- OneRowRelation + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query analysis +DropVariable false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query analysis +CreateSQLFunctionCommand test_udf, IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING, INT, IDENTIFIER('param1') + length(IDENTIFIER('param2')), false, true, false, false + + +-- !query +SELECT test_udf(5, 'hello') +-- !query analysis +Project [test_udf(param1#x, param2#x) AS test_udf(5, hello)#x] ++- Project [cast(5 as int) AS param1#x, cast(hello as string) AS param2#x] + +- OneRowRelation + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query analysis +DropFunctionCommand test_udf, false, true + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query analysis +CreateSQLFunctionCommand test_table_udf, IDENTIFIER('input_val') INT, IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING, SELECT IDENTIFIER('input_val'), 'result', true, true, false, false + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query analysis +Project [col1#x, col2#x] ++- SQLFunctionNode test_table_udf + +- SubqueryAlias test_table_udf + +- Project [cast(input_val#x as int) AS col1#x, cast(result#x as string) AS col2#x] + +- Project [cast(42 as int) AS input_val#x, result AS result#x] + +- OneRowRelation + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query analysis +DropFunctionCommand test_table_udf, false, true + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query analysis +Project [c1#x.c2 AS c1.c2#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, false + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query analysis +Project [c1#x, c2#x] ++- Filter (c1#x = 1) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, false + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2), [c1, c3] ++- Project [col1#x AS c1#x, col2#x AS c3#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST, c1#x ASC NULLS FIRST, c3#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x, c1#x, c3#x] + +- Project [c1#x, c2#x, c3#x, c1#x] + +- Join Inner, (c1#x = c1#x) + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + : +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test2 + +- Relation spark_catalog.identifier_clause_test_schema.integration_test2[c1#x,c3#x] csv + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query analysis +Project [c1#x, c2#x, rn#x] ++- Project [c1#x, c2#x, rn#x, rn#x] + +- Window [row_number() windowspecdefinition(c2#x, c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#x], [c2#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query analysis +Sort [c2#x ASC NULLS FIRST, count(c1)#xL ASC NULLS FIRST], true ++- Aggregate [c2#x], [c2#x, count(c1#x) AS count(c1)#xL] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query analysis +Sort [c1#x DESC NULLS LAST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query analysis +CommandResult Execute InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] + +- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] + +- Project [c1#x AS c1#x, c2#x AS c2#x] + +- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`my_table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 115, + "stopIndex" : 144, + "fragment" : "IDENTIFIER(:schema '.' :table)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias my_cte +: +- Project [col1#x AS c1#x] +: +- LocalRelation [col1#x] ++- Project [c1#x] + +- SubqueryAlias my_cte + +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query analysis +CommandResult Execute CreateViewCommand + +- CreateViewCommand `test_view`, [(test_col,None)], VALUES(1), false, true, LocalTempView, UNSUPPORTED, true + +- LocalRelation [col1#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query analysis +Project [test_col#x] ++- SubqueryAlias test_view + +- View (`test_view`, [test_col#x]) + +- Project [cast(col1#x as int) AS test_col#x] + +- LocalRelation [col1#x] + + +-- !query +DROP VIEW test_view +-- !query analysis +DropTempViewCommand test_view + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query analysis +CommandResult Execute AlterTableAddColumnsCommand + +- AlterTableAddColumnsCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, [StructField(c4,IntegerType,true)] + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`identifier_clause_test_schema`.`integration_test`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query analysis +Project [map(mykey, 42)[mykey] AS result#x] ++- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`.`c1`", + "proposal" : "`IDENTIFIER('t')`.`c1`, `IDENTIFIER('t')`.`c2`, `IDENTIFIER('t')`.`c4`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "IDENTIFIER(:alias '.c1')" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query analysis +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x,c4#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST, c4#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x, c4#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x,c4#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`col1`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test + + +-- !query +DROP TABLE integration_test2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test2 + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, false + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test), [arr] ++- Project [col1#x AS arr#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query analysis +Sort [arr#x ASC NULLS FIRST, col#x ASC NULLS FIRST], true ++- Project [arr#x, col#x] + +- Generate explode(arr#x), false, tbl, [col#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.lateral_test + +- Relation spark_catalog.identifier_clause_test_schema.lateral_test[arr#x] parquet + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query analysis +Sort [arr#x ASC NULLS FIRST, my_col#x ASC NULLS FIRST], true ++- Project [arr#x, my_col#x] + +- Generate explode(arr#x), true, my_table, [my_col#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.lateral_test + +- Relation spark_catalog.identifier_clause_test_schema.lateral_test[arr#x] parquet + + +-- !query +DROP TABLE lateral_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.lateral_test + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, false + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test), [id, a, b, c] ++- Project [col1#x AS id#x, col2#x AS a#x, col3#x AS b#x, col4#x AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query analysis +Sort [id#x ASC NULLS FIRST, c#x ASC NULLS FIRST, col#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [id#x, c#x, col#x, val#x] + +- Filter isnotnull(coalesce(val#x)) + +- Expand [[id#x, c#x, col_a, a#x], [id#x, c#x, col_b, b#x]], [id#x, c#x, col#x, val#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.unpivot_test + +- Relation spark_catalog.identifier_clause_test_schema.unpivot_test[id#x,a#x,b#x,c#x] csv + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query analysis +Sort [id#x ASC NULLS FIRST, col#x ASC NULLS FIRST, v1#x ASC NULLS FIRST, v2#x ASC NULLS FIRST], true ++- Project [id#x, col#x, v1#x, v2#x] + +- Filter isnotnull(coalesce(v1#x, v2#x)) + +- Expand [[id#x, cols_ab, a#x, b#x], [id#x, cols_bc, b#x, c#x]], [id#x, col#x, v1#x, v2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.unpivot_test + +- Relation spark_catalog.identifier_clause_test_schema.unpivot_test[id#x,a#x,b#x,c#x] csv + + +-- !query +DROP TABLE unpivot_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, false + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.hint_test + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'FROM'", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 17fa47be4eec3..16e1850d5e59f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -289,3 +289,11 @@ EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP('key1', 'valu -- !query EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP(1, 'one', 2, 'two') AS p; +-- !query +-- Test unbound parameter markers without USING clause +-- named parameter without USING clause should fail +EXECUTE IMMEDIATE 'SELECT :param'; + +-- !query +-- positional parameter without USING clause should fail +EXECUTE IMMEDIATE 'SELECT ?'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql new file mode 100644 index 0000000000000..ae1f10f1af1f2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.identifierClause = true +--IMPORT identifier-clause.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index 4aa8019097fdf..c90165d31e90d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -157,7 +157,6 @@ SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T'); WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC'); --- Not supported SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')); SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1); @@ -168,10 +167,216 @@ SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab'); SELECT * FROM IDENTIFIER('s').tab; SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1); +SELECT 1 AS IDENTIFIER('col1'); +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')); WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v); -INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1); +SELECT c1 FROM v; CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV; +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); +SELECT c1 FROM tab; +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1'); +SELECT col1 FROM tab; +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT; +SELECT c2 FROM tab; +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2'); +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed'); +SELECT * FROM tab_renamed; +DROP TABLE IF EXISTS tab_renamed; +DROP TABLE IF EXISTS tab; + +-- Error because qualified names are not allowed +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV; +DROP TABLE IF EXISTS test_col_with_dot; +-- Identifier-lite: table alias with qualified name should error (table alias must be single) +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2); +-- Identifier-lite: column alias with qualified name should error (column alias must be single) +SELECT 1 AS IDENTIFIER('col1.col2'); + +-- Additional coverage: SHOW commands with identifier-lite +CREATE SCHEMA identifier_clause_test_schema; +USE identifier_clause_test_schema; +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV; +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema'); +SHOW PARTITIONS IDENTIFIER('test_show'); +SHOW CREATE TABLE IDENTIFIER('test_show'); +DROP TABLE test_show; + +-- SET CATALOG with identifier-lite +-- SET CATALOG IDENTIFIER('spark_catalog'); + +-- DESCRIBE with different forms +CREATE TABLE test_desc(c1 INT) USING CSV; +DESCRIBE TABLE IDENTIFIER('test_desc'); +DESCRIBE FORMATTED IDENTIFIER('test_desc'); +DESCRIBE EXTENDED IDENTIFIER('test_desc'); +DESC IDENTIFIER('test_desc'); +DROP TABLE test_desc; + +-- COMMENT ON COLUMN with identifier-lite +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV; +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment'; +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment'; +DROP TABLE test_comment; + +-- Additional identifier tests with qualified table names in various commands +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV; +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS; +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table'); +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table'); +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table'); +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table'); + +-- Session variables with identifier-lite +DECLARE IDENTIFIER('my_var') = 'value'; +SET VAR IDENTIFIER('my_var') = 'new_value'; +SELECT IDENTIFIER('my_var'); +DROP TEMPORARY VARIABLE IDENTIFIER('my_var'); + +-- SQL UDF with identifier-lite in parameter names and return statement +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')); + +SELECT test_udf(5, 'hello'); +DROP TEMPORARY FUNCTION test_udf; + +-- SQL UDF with table return type using identifier-lite +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result'; + +SELECT * FROM test_table_udf(42); +DROP TEMPORARY FUNCTION test_table_udf; + +-- Integration tests: Combining parameter markers, string coalescing, and IDENTIFIER +-- These tests demonstrate the power of combining IDENTIFIER with parameters + +-- Test 1: IDENTIFIER with parameter marker for table name +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab; + +-- Test 2: IDENTIFIER with string coalescing for column name +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1; + +-- Test 3: IDENTIFIER with parameter and string literal coalescing for qualified table name +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV; +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b'); +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table; + +-- Test 4: IDENTIFIER in column reference with parameter and string coalescing +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix; + +-- Test 5: IDENTIFIER in WHERE clause with parameters +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val; + +-- Test 6: IDENTIFIER in JOIN with parameters for table and column names +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV; +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y'); +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col; + +-- Test 7: IDENTIFIER in window function with parameter for partition column +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord; + +-- Test 8: IDENTIFIER in aggregate function with string coalescing +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col; + +-- Test 9: IDENTIFIER in ORDER BY with multiple parameters +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2; + +-- Test 10: IDENTIFIER in INSERT with parameter for column name +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2; + +-- Test 11: Complex - IDENTIFIER with nested string operations +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias; + +-- Test 12: IDENTIFIER in CTE name with parameter +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name; + +-- Test 13: IDENTIFIER in view name with parameter +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name; +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view; +DROP VIEW test_view; + +-- Test 14: IDENTIFIER in ALTER TABLE with parameters +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col; +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col; + +-- Test 15: IDENTIFIER with dereference using parameters +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val; + +-- Test 16: IDENTIFIER in table alias with string coalescing +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias; + +-- Test 17: Multiple IDENTIFIER clauses with different parameter combinations +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab; + +-- Test 19: IDENTIFIER with qualified name coalescing for schema.table.column pattern +-- This should work for multi-part identifiers +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias; + +-- Test 20: Error case - IDENTIFIER with too many parts from parameter coalescing +-- This should error as column alias must be single identifier +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col; + +-- Cleanup +DROP TABLE integration_test; +DROP TABLE integration_test2; + +-- LATERAL VIEW with IDENTIFIER() for table and column names +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET; +INSERT INTO lateral_test VALUES (array(1, 2, 3)); +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL; +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL; +DROP TABLE lateral_test; + +-- UNPIVOT with IDENTIFIER() for value column alias +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV; +INSERT INTO unpivot_test VALUES (1, 10, 20, 30); +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL; +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL; +DROP TABLE unpivot_test; + +-- All the following tests fail because they are not about "true" identifiers + +-- This should fail - named parameters don't support IDENTIFIER() +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1); + +-- Hint names use simpleIdentifier - these should fail +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV; +INSERT INTO hint_test VALUES (1, 2), (3, 4); +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test; +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test; +DROP TABLE hint_test; + +-- These should fail - function scope doesn't support IDENTIFIER() +SHOW IDENTIFIER('USER') FUNCTIONS; +-- EXTRACT field name uses simpleIdentifier - should fail +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15'); +-- TIMESTAMPADD unit is a token, not identifier - should fail +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15'); +DROP SCHEMA identifier_clause_test_schema; diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 06adf4435046a..dd1207b4f2be5 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -1211,3 +1211,47 @@ EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP(1, 'one', 2, struct> -- !query output map {1:"one",2:"two"} + + +-- !query +EXECUTE IMMEDIATE 'SELECT :param' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "param" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : ":param" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT ?' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "_7" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "?" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out new file mode 100644 index 0000000000000..ad351074692f5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -0,0 +1,2736 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET hivevar:colname = 'c' +-- !query schema +struct +-- !query output +hivevar:colname 'c' + + +-- !query +SELECT IDENTIFIER(${colname} || '_1') FROM VALUES(1) AS T(c_1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('t.c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('`t`.c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('`c 1`') FROM VALUES(1) AS T(`c 1`) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('``') FROM VALUES(1) AS T(``) +-- !query schema +struct<:int> +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE SCHEMA IF NOT EXISTS s +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE s.tab(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +USE SCHEMA s +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DELETE FROM IDENTIFIER('ta' || 'b') WHERE 1=0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "DELETE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s.c1 = t.c1 + WHEN MATCHED THEN UPDATE SET c1 = 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('tab') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('s.tab') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('`s`.`tab`') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('t' || 'a' || 'b') +-- !query schema +struct +-- !query output +1 + + +-- !query +USE SCHEMA default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE s.tab +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA s +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +USE identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "REPLACE TABLE", + "tableName" : "`spark_catalog`.`identifier_clauses`.`tab`" + } +} + + +-- !query +CACHE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +UNCACHE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tab(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tab VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +ANALYZE TABLE IDENTIFIER('ta' || 'b') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE IDENTIFIER('ta' || 'b') ADD COLUMN c2 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TBLPROPERTIES IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output +c1 +c2 + + +-- !query +COMMENT ON TABLE IDENTIFIER('ta' || 'b') IS 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +REPAIR TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_PARTITIONED_TABLE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "MSCK REPAIR TABLE", + "tableIdentWithDB" : "`spark_catalog`.`default`.`tab`" + } +} + + +-- !query +TRUNCATE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +ALTER VIEW IDENTIFIER('v') AS VALUES(2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET PROPERTIES (somekey = 'somevalue') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET LOCATION 'someloc' +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON SCHEMA IDENTIFIER('id' || 'ent') IS 'some comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct +-- !query output +Catalog Name spark_catalog +Comment some comment +Location [not included in comparison]/{warehouse_dir}/someloc +Namespace Name ident +Owner [not included in comparison] + + +-- !query +SHOW TABLES IN IDENTIFIER('id' || 'ent') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW TABLE EXTENDED IN IDENTIFIER('id' || 'ent') LIKE 'hello' +-- !query schema +struct +-- !query output + + + +-- !query +USE IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CURRENT SCHEMA +-- !query schema +struct +-- !query output +spark_catalog ident + + +-- !query +USE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +USE SCHEMA default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA ident +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct +-- !query output +Class: test.org.apache.spark.sql.MyDoubleAvg +Function: spark_catalog.ident.mydoubleavg +Usage: N/A. + + +-- !query +REFRESH FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA ident +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE var = 'sometable' +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER(var)(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR var = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER(var) FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VAR var = 'some' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IDENTIFIER(var || 'table') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('c 1') FROM VALUES(1) AS T(`c 1`) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : ": extra input '1'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "IDENTIFIER('c 1')" + } ] +} + + +-- !query +SELECT IDENTIFIER('') FROM VALUES(1) AS T(``) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_EMPTY_STATEMENT", + "sqlState" : "42617", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "IDENTIFIER('')" + } ] +} + + +-- !query +VALUES(IDENTIFIER(CAST(NULL AS STRING))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NULL", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "CAST(NULL AS STRING)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 38, + "fragment" : "CAST(NULL AS STRING)" + } ] +} + + +-- !query +VALUES(IDENTIFIER(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "substr('HELLO', 1, CAST((rand() + CAST(1 AS DOUBLE)) AS INT))", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 48, + "fragment" : "SUBSTR('HELLO', 1, RAND() + 1)" + } ] +} + + +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 25, + "fragment" : "1" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP TABLE IDENTIFIER('a.b.c') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP VIEW IDENTIFIER('a.b.c') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +COMMENT ON TABLE IDENTIFIER('a.b.c.d') IS 'hello' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +VALUES(IDENTIFIER(1)()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER('a.b.c.d')()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "IDENTIFIER('a.b.c.d')()" + } ] +} + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + "sqlState" : "42000", + "messageParameters" : { + "database" : "`default`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 108, + "fragment" : "CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + "sqlState" : "42000", + "messageParameters" : { + "name" : "`default`.`myDoubleAvg`", + "statement" : "DROP TEMPORARY FUNCTION" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + "sqlState" : "428EK", + "messageParameters" : { + "actualName" : "`default`.`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1)" + } ] +} + + +-- !query +create temporary view identifier('v1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +cache table identifier('t1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table identifier('t2') using csv as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into identifier('t2') select my_col from (values (3) as (my_col)) group by 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop view v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE agg = 'max' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE col = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE tab = 'T' +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query schema +struct +-- !query output +c + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query schema +struct +-- !query output +c + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''x.win''", + "hint" : "" + } +} + + +-- !query +SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "IDENTIFIER('t')" + } ] +} + + +-- !query +SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT * FROM s.IDENTIFIER('tab') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''win''", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'AS'" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''my_table''", + "hint" : "" + } +} + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''v''", + "hint" : "" + } +} + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c1 FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 16, + "fragment" : "v" + } ] +} + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT col1 FROM tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 20, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c2 FROM tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM tab_renamed +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab_renamed`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "tab_renamed" + } ] +} + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''schema.table''", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1.col2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE SCHEMA identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +USE identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" + } +} + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.identifier_clause_test_schema.test_show ( + c1 INT, + c2 STRING) +USING CSV + + +-- !query +DROP TABLE test_show +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DROP TABLE test_desc +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE test_comment +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing EQ" + } +} + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query schema +struct +-- !query output +value + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT test_udf(5, 'hello') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "test_udf(5, 'hello')" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`" + } +} + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "sqlState" : "42883", + "messageParameters" : { + "name" : "`test_table_udf`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 32, + "fragment" : "test_table_udf(42)" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_table_udf`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query schema +struct +-- !query output +42 + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query schema +struct +-- !query output +1 a + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 103, + "fragment" : "SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query schema +struct +-- !query output +1 a 1 +2 b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query schema +struct +-- !query output +a 1 +b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query schema +struct +-- !query output +2 b +1 a + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 145, + "fragment" : "SELECT IDENTIFIER(concat(:schema, '.', :table, '.c1')) FROM VALUES(named_struct('c1', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema '.' :table))" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`test_view`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 46, + "fragment" : "IDENTIFIER(:view)" + } ] +} + + +-- !query +DROP VIEW test_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`identifier_clause_test_schema`.`test_view`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT map(:key, :val).IDENTIFIER(:key) AS result" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT IDENTIFIER(:alias '.c1') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "SELECT 1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE integration_test2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE lateral_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE unpivot_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 596745b4ba5d8..4a0f9abe56394 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -118,9 +118,11 @@ struct<> -- !query output org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "UPDATE TABLE" + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -133,9 +135,11 @@ struct<> -- !query output org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "MERGE INTO TABLE" + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -980,7 +984,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", @@ -1175,29 +1180,28 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "''x.win''", - "hint" : "" - } + "identifier" : "`x`.`win`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 44, + "fragment" : "IDENTIFIER('x.win')" + } ] } -- !query SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +1 -- !query @@ -1226,33 +1230,17 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +1 -- !query SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +1 -- !query @@ -1260,12 +1248,12 @@ SELECT * FROM s.IDENTIFIER('tab') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", - "sqlState" : "42000", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "funcName" : "`s`.`IDENTIFIER`" + "relationName" : "`s`.`tab`" }, "queryContext" : [ { "objectType" : "", @@ -1282,14 +1270,20 @@ SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "IDENTIFIER('s').IDENTIFIER('tab')" + } ] } @@ -1298,103 +1292,1049 @@ SELECT * FROM IDENTIFIER('s').tab -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "IDENTIFIER('s').tab" + } ] } -- !query SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) -- !query schema +struct +-- !query output +1 + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException + + + +-- !query +SELECT c1 FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct +-- !query output +1 + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''win''", - "hint" : "" + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +SELECT col1 FROM tab -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "error" : "'WINDOW'", - "hint" : "" - } + "objectName" : "`col1`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "col1" + } ] } -- !query -WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException + + + +-- !query +SELECT c2 FROM tab +-- !query schema +struct +-- !query output +NULL + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''v''", - "hint" : "" + "operation" : "DROP COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM tab_renamed +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) -- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : ": missing ')'" - } + "identifier" : "`schema`.`table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 65, + "fragment" : "VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2)" + } ] } -- !query -CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +SELECT 1 AS IDENTIFIER('col1.col2') -- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : "" - } + "identifier" : "`col1`.`col2`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "1 AS IDENTIFIER('col1.col2')" + } ] } -- !query -CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +CREATE SCHEMA identifier_clause_test_schema -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException + + + +-- !query +USE identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" + } +} + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.identifier_clause_test_schema.test_show ( + c1 INT, + c2 STRING) +USING CSV + + +-- !query +DROP TABLE test_show +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DROP TABLE test_desc +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE test_comment +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query schema +struct +-- !query output +new_value + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT test_udf(5, 'hello') +-- !query schema +struct +-- !query output +10 + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query schema +struct +-- !query output +42 result + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query schema +struct +-- !query output +42 + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query schema +struct +-- !query output +1 a + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query schema +struct +-- !query output +1 a 1 x +2 b 2 y + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query schema +struct +-- !query output +1 a 1 +2 b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query schema +struct +-- !query output +a 1 +b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query schema +struct +-- !query output +2 b +1 a + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`my_table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 115, + "stopIndex" : 144, + "fragment" : "IDENTIFIER(:schema '.' :table)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW test_view +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`identifier_clause_test_schema`.`integration_test`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query schema +struct +-- !query output +42 + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`.`c1`", + "proposal" : "`IDENTIFIER('t')`.`c1`, `IDENTIFIER('t')`.`c2`, `IDENTIFIER('t')`.`c4`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "IDENTIFIER(:alias '.c1')" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query schema +struct +-- !query output +1 a +2 b +3 c + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query schema +struct +-- !query output +1 a NULL +2 b NULL +3 c NULL + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`col1`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE integration_test2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query schema +struct,col:int> +-- !query output +[1,2,3] 1 +[1,2,3] 2 +[1,2,3] 3 + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query schema +struct,my_col:int> +-- !query output +[1,2,3] 1 +[1,2,3] 2 +[1,2,3] 3 + + +-- !query +DROP TABLE lateral_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query schema +struct +-- !query output +1 30 col_a 10 +1 30 col_b 20 + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query schema +struct +-- !query output +1 cols_ab 10 20 +1 cols_bc 20 30 + + +-- !query +DROP TABLE unpivot_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException { "errorClass" : "PARSE_SYNTAX_ERROR", "sqlState" : "42601", @@ -1403,3 +2343,50 @@ org.apache.spark.sql.catalyst.parser.ParseException "hint" : "" } } + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'FROM'", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index e30b48fdb1764..40cfb54159d59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -2374,4 +2374,76 @@ class ParametersSuite extends QueryTest with SharedSparkSession { expectedStopPos = Some(46) // End of "nonexistent_table" in inner query ) } + + test("detect unbound named parameter with empty map") { + // When sql() is called with empty map, parameter markers should still be detected + val exception = intercept[AnalysisException] { + spark.sql("SELECT :param", Map.empty[String, Any]) + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "param"), + context = ExpectedContext( + fragment = ":param", + start = 7, + stop = 12)) + } + + test("detect unbound positional parameter with empty array") { + // When sql() is called with empty array, parameter markers should still be detected + val exception = intercept[AnalysisException] { + spark.sql("SELECT ?", Array.empty[Any]) + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "_7"), + context = ExpectedContext( + fragment = "?", + start = 7, + stop = 7)) + } + + test("detect unbound named parameter with no arguments") { + val exception = intercept[AnalysisException] { + spark.sql("SELECT :param") + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "param"), + context = ExpectedContext( + fragment = ":param", + start = 7, + stop = 12)) + } + + test("detect unbound positional parameter with no arguments") { + val exception = intercept[AnalysisException] { + spark.sql("SELECT ?") + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "_7"), + context = ExpectedContext( + fragment = "?", + start = 7, + stop = 7)) + } + + test("empty map with no parameters - should succeed") { + // When there are no parameter markers, empty map should work fine + checkAnswer( + spark.sql("SELECT 1", Map.empty[String, Any]), + Row(1)) + } + + test("empty array with no parameters - should succeed") { + // When there are no parameter markers, empty array should work fine + checkAnswer( + spark.sql("SELECT 1", Array.empty[Any]), + Row(1)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala index ea305f219aa50..e98cfc8bebccf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala @@ -867,4 +867,41 @@ class StringLiteralCoalescingSuite extends QueryTest with SharedSparkSession { Row("1a2b3c4") ) } + + test("parameter substitution with quote spacing - legacy consecutive string literals disabled") { + // With LEGACY_CONSECUTIVE_STRING_LITERALS enabled, '' would normally produce a single quote + // But with parameter substitution, 'literal':param should insert a space to prevent + // the closing quote and opening quote from being interpreted as an escape sequence + withSQLConf("spark.sql.legacy.consecutiveStringLiterals.enabled" -> "false") { + checkAnswer( + spark.sql("SELECT 'hello':p, 'hello''world'", Map("p" -> "world")), + // Space parameter separates literals, no singleton quote in middle + Row("helloworld", "hello'world") + ) + } + } + + test("parameter substitution with quote spacing - legacy consecutive string literals enabled") { + withSQLConf("spark.sql.legacy.consecutiveStringLiterals.enabled" -> "true") { + checkAnswer( + spark.sql("SELECT 'hello':p, 'hello''world'", Map("p" -> "world")), + // Space parameter separates literals, no singleton quote in middle + Row("helloworld", "helloworld") + ) + } + } + + // ======================================================================== + // Legacy Mode Tests - JSON Path Expressions vs Parameter Substitution + // ======================================================================== + test("JSON path expression - new mode with parameter substitution") { + // In new mode (constantsOnly=false), :name is a parameter marker when args provided + withSQLConf("spark.sql.legacy.parameterSubstitution.constantsOnly" -> "false") { + // The :name gets substituted with the parameter value + checkAnswer( + spark.sql("SELECT '{\"name\":\"joe\"}' :name", Map("name" -> "replaced")), + Row("{\"name\":\"joe\"}replaced") + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index bf10d50db1423..6f79e62a3a680 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2312,8 +2312,10 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[SparkUnsupportedOperationException] { sql(s"UPDATE $t SET name='Robert', age=32 WHERE p=1") }, - condition = "_LEGACY_ERROR_TEMP_2096", - parameters = Map("ddl" -> "UPDATE TABLE") + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "operation" -> "UPDATE TABLE") ) } } @@ -2418,8 +2420,10 @@ class DataSourceV2SQLSuiteV1Filter |WHEN MATCHED AND (target.p > 0) THEN UPDATE SET * |WHEN NOT MATCHED THEN INSERT *""".stripMargin) }, - condition = "_LEGACY_ERROR_TEMP_2096", - parameters = Map("ddl" -> "MERGE INTO TABLE")) + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`target`", + "operation" -> "MERGE INTO TABLE")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 9e16c5b2a3ddd..45a656f7a6d5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -720,7 +720,7 @@ class QueryCompilationErrorsSuite }, condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", - parameters = Map("identifier" -> "`db_name`.`schema_name`.`view_name`") + parameters = Map("identifier" -> "`db_name`.`schema_name`.`view_name`", "limit" -> "2") ) } @@ -741,7 +741,7 @@ class QueryCompilationErrorsSuite }, condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", - parameters = Map("identifier" -> "`db_name`.`schema_name`.`new_table_name`") + parameters = Map("identifier" -> "`db_name`.`schema_name`.`new_table_name`", "limit" -> "2") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index e76a8556230af..3f25a1e139faa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -1163,7 +1163,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { checkError( exception = intercept[AnalysisException](testIdent.asTableIdentifier), condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", - parameters = Map("identifier" -> "`a`.`b`.`c`") + parameters = Map("identifier" -> "`a`.`b`.`c`", "limit" -> "2") ) } @@ -1172,7 +1172,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { checkError( exception = intercept[AnalysisException](testIdent.asFunctionIdentifier), condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", - parameters = Map("identifier" -> "`a`.`b`.`c`") + parameters = Map("identifier" -> "`a`.`b`.`c`", "limit" -> "2") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index c6a48d98fa980..fd88559d4f988 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -3078,7 +3078,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel }, condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", - parameters = Map("identifier" -> "`db_name`.`schema_name`.`function_name`") + parameters = Map("identifier" -> "`db_name`.`schema_name`.`function_name`", "limit" -> "2") ) } finally { JdbcDialects.unregisterDialect(testH2Dialect) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala index db561d95058e3..951be21839d21 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala @@ -274,4 +274,66 @@ class SqlScriptingE2eSuite extends QueryTest with SharedSparkSession { condition = "INVALID_QUERY_MIXED_QUERY_PARAMETERS", parameters = Map()) } + + test("SQL Script labels with identifier") { + val sqlScript = + """ + |BEGIN + | IDENTIFIER('loop_label'): LOOP + | SELECT 1; + | LEAVE IDENTIFIER('loop_label'); + | END LOOP IDENTIFIER('loop_label'); + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } + + test("SQL Script with labeled BEGIN/END block using identifier") { + val sqlScript = + """ + |BEGIN + | IDENTIFIER('block_label'): BEGIN + | DECLARE IDENTIFIER('x') INT DEFAULT 1; + | SELECT x; + | END IDENTIFIER('block_label'); + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } + + test("WHILE loop with identifier label") { + val sqlScript = + """ + |BEGIN + | DECLARE counter INT DEFAULT 0; + | IDENTIFIER('while_label'): WHILE counter < 3 DO + | SET IDENTIFIER('counter') = counter + 1; + | END WHILE IDENTIFIER('while_label'); + | SELECT counter; + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(3))) + } + + test("REPEAT loop with identifier label") { + val sqlScript = + """ + |BEGIN + | DECLARE cnt INT DEFAULT 0; + | repeat_label: REPEAT + | SET cnt = cnt + 1; + | UNTIL cnt >= 2 + | END REPEAT IDENTIFIER('repeat_label'); + | SELECT cnt; + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(2))) + } + + test("FOR loop with identifier") { + val sqlScript = + """ + |BEGIN + | IDENTIFIER('for_label'): FOR IDENTIFIER('row') AS SELECT 1 AS c1 DO + | SELECT row.c1; + | END FOR for_label; + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } } From 8c1adc07f563c7a0c615281a8fce70e499ca69f4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 13 Nov 2025 09:21:53 -0800 Subject: [PATCH 125/400] [SPARK-54194][PYTHON][FOLLOWUP] Fix `connectutils.py` to import `pb2` conditionally ### What changes were proposed in this pull request? This PR is a follow-up of the following to fix `connectutils.py` to import `pb2` conditionally. - #52894 ### Why are the changes needed? Currently, Python CIs are broken like the following. - https://github.com/apache/spark/actions/workflows/build_python_3.11_classic_only.yml - https://github.com/apache/spark/actions/runs/19316448951/job/55248810741 - https://github.com/apache/spark/actions/workflows/build_python_3.12.yml - https://github.com/apache/spark/actions/runs/19275741458/job/55212353468 ``` File "/__w/spark/spark/python/pyspark/testing/connectutils.py", line 26, in import pyspark.sql.connect.proto as pb2 File "/__w/spark/spark/python/pyspark/sql/connect/proto/__init__.py", line 18, in from pyspark.sql.connect.proto.base_pb2_grpc import * File "/__w/spark/spark/python/pyspark/sql/connect/proto/base_pb2_grpc.py", line 19, in import grpc ModuleNotFoundError: No module named 'grpc' ``` ### Does this PR introduce _any_ user-facing change? No behavior change. We has been importing `pyspark.sql.connect` conditionally before #52894 . ### How was this patch tested? Pass the CIs and manual test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53037 from dongjoon-hyun/SPARK-54194. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 63bcc871bdec31cbc717dc83b54d76d6796c16bb) Signed-off-by: Dongjoon Hyun --- python/pyspark/testing/connectutils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index d895c1d8a26b1..302ae2a5751f7 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -23,7 +23,6 @@ import uuid import contextlib -import pyspark.sql.connect.proto as pb2 from pyspark import Row, SparkConf from pyspark.util import is_remote_only from pyspark.testing.utils import PySparkErrorTestUtils @@ -53,6 +52,7 @@ from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.connect.plan import Read, Range, SQL, LogicalPlan from pyspark.sql.connect.session import SparkSession + import pyspark.sql.connect.proto as pb2 class MockRemoteSession: From ef8d9cb4bfd232ab7e60a891dcb97264e16dff44 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Thu, 13 Nov 2025 09:33:48 -0800 Subject: [PATCH 126/400] [SPARK-54333][BUILD] Upgrade `commons-io` to 2.21.0 ### What changes were proposed in this pull request? This pr aims to upgrade Apache `commons-io` from 2.20.0 to 2.21.0. ### Why are the changes needed? The new version has passed the testing for Java 24 and comes with some bug fixes. The release notes are as follows: - https://commons.apache.org/proper/commons-io/changes.html#a2.21.0 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #53034 from LuciferYang/SPARK-54333. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun (cherry picked from commit 5e46f4c5fec48891d094f8a639176c194b408be0) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 5d619db2a75e9..55b63d8a67930 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -42,7 +42,7 @@ commons-compiler/3.1.9//commons-compiler-3.1.9.jar commons-compress/1.28.0//commons-compress-1.28.0.jar commons-crypto/1.1.0//commons-crypto-1.1.0.jar commons-dbcp/1.4//commons-dbcp-1.4.jar -commons-io/2.20.0//commons-io-2.20.0.jar +commons-io/2.21.0//commons-io-2.21.0.jar commons-lang/2.6//commons-lang-2.6.jar commons-lang3/3.19.0//commons-lang3-3.19.0.jar commons-math3/3.6.1//commons-math3-3.6.1.jar diff --git a/pom.xml b/pom.xml index 073ecf08b2e17..3d7f9a0874831 100644 --- a/pom.xml +++ b/pom.xml @@ -190,7 +190,7 @@ 3.0.4 1.19.0 1.28.0 - 2.20.0 + 2.21.0 2.6 From 0c0390468ce3d73f54a04f15885c99f39e4847f2 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Thu, 13 Nov 2025 12:04:34 -0800 Subject: [PATCH 127/400] [SPARK-54336][SQL] Fix `BloomFilterMightContain` input type check with `ScalarSubqueryReference` ### What changes were proposed in this pull request? This PR fixes the case when a scalar subquery with a `BloomFilterMightContain` expression is added to `PlanMerger`, but its `bloomFilterExpression` contains a merged scalar subquery which was replaced to a `ScalarSubqueryReference` before. In this case `BloomFilterMightContain` was unresolved due to its `checkInputDataTypes()` failure. ### Why are the changes needed? To ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53038 from peter-toth/SPARK-54336-fix-bloomfiltermightcontain-type-check. Authored-by: Peter Toth Signed-off-by: Dongjoon Hyun (cherry picked from commit 06b3d4c4e7166c3107c7cc95ff0da2d9f256eabd) Signed-off-by: Dongjoon Hyun --- .../expressions/BloomFilterMightContain.scala | 2 ++ .../sql/BloomFilterAggregateQuerySuite.scala | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala index 784bea899c4c8..e3ff7c5f05f0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.Cast.{toSQLExpr, toSQLId, toSQLType} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral} import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper +import org.apache.spark.sql.catalyst.optimizer.ScalarSubqueryReference import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE import org.apache.spark.sql.types._ import org.apache.spark.util.sketch.BloomFilter @@ -58,6 +59,7 @@ case class BloomFilterMightContain( case GetStructField(subquery: PlanExpression[_], _, _) if !subquery.containsPattern(OUTER_REFERENCE) => TypeCheckResult.TypeCheckSuccess + case _: ScalarSubqueryReference => TypeCheckResult.TypeCheckSuccess case _ => DataTypeMismatch( errorSubClass = "BLOOM_FILTER_BINARY_OP_WRONG_TYPE", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala index fb279b1db6fc9..0a8e31b6bac4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala @@ -367,4 +367,23 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { checkNumBits(100, 2935) checkNumBits(1, 38) } + + test("SPARK-54336: Fix BloomFilterMightContain type check with ScalarSubqueryReference") { + val table = "bloom_filter_test" + withTempView(table) { + Seq(0).toDF("col").createOrReplaceTempView(table) + val df = sql( + s""" + |SELECT + | (SELECT + | first(might_contain( + | (SELECT bloom_filter_agg(col) FROM $table), + | 0L + | )) + | FROM $table) + |FROM $table + |""".stripMargin) + checkAnswer(df, Row(true)) + } + } } From 32a88626e13642015d3387f4f79192ed9d7a1dfe Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 13 Nov 2025 12:35:28 -0800 Subject: [PATCH 128/400] [SPARK-54331][PYTHON][TESTS] Optimize `pyspark.sql.tests.connect.test_connect_plan` ### What changes were proposed in this pull request? Optimize `pyspark.sql.tests.connect.test_connect_plan`, by removing the remote session creation ### Why are the changes needed? Recently, I notice that this test become extremely slow, e.g. in https://github.com/apache/spark/actions/runs/19318125464/job/55253739424 before ``` Starting test(python3.11): pyspark.sql.tests.connect.test_connect_plan (temp output: /__w/spark/spark/python/target/763913e5-9ba0-46ed-a583-56bf5fa5f588/python3.11__pyspark.sql.tests.connect.test_connect_plan__nbrkokah.log) Finished test(python3.11): pyspark.sql.tests.connect.test_connect_plan (1222s) ``` after ``` Starting test(python3.11): pyspark.sql.tests.connect.test_connect_plan (temp output: /__w/spark/spark/python/target/45831997-66c0-4c44-89cf-1ce85dc89ee7/python3.11__pyspark.sql.tests.connect.test_connect_plan__tyaymd_b.log) Finished test(python3.11): pyspark.sql.tests.connect.test_connect_plan (1s) ``` The tests theirselves are pretty fast, so I think the root cause is the remote session creation which is not necessary in this test which is for validation of protobufs ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #53032 from zhengruifeng/opt_test_connect_plan. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 602a4bd948815b7d5ad25877c39dd7d3a4837f2c) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/connect/test_connect_plan.py | 4 ++-- python/pyspark/testing/connectutils.py | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_plan.py b/python/pyspark/sql/tests/connect/test_connect_plan.py index 1d4d85e8426ee..3f785d4ee7130 100644 --- a/python/pyspark/sql/tests/connect/test_connect_plan.py +++ b/python/pyspark/sql/tests/connect/test_connect_plan.py @@ -753,7 +753,7 @@ def test_column_regexp(self): self.assertIsInstance(col, Column) self.assertEqual("Column<'UnresolvedRegex(col_name)'>", str(col)) - col_plan = col.to_plan(self.session.client) + col_plan = col.to_plan(None) self.assertIsNotNone(col_plan) self.assertEqual(col_plan.unresolved_regex.col_name, "col_name") @@ -937,7 +937,7 @@ def test_column_alias(self) -> None: self.assertEqual("Column<'a AS martin'>", str(col0)) col0 = col("a").alias("martin", metadata={"pii": True}) - plan = col0.to_plan(self.session.client) + plan = col0.to_plan(None) self.assertIsNotNone(plan) self.assertEqual(plan.alias.metadata, '{"pii": true}') diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index 302ae2a5751f7..8539e16f03fc1 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -16,7 +16,6 @@ # import shutil import tempfile -import typing import os import functools import unittest @@ -133,7 +132,6 @@ def _with_plan(cls, plan): @classmethod def setUpClass(cls): cls.connect = MockRemoteSession() - cls.session = SparkSession.builder.remote().getOrCreate() cls.tbl_name = "test_connect_plan_only_table_1" cls.connect.set_hook("readTable", cls._read_table) From ecf2ea545ee1f4d53e295c5f2bb38c27f3268f83 Mon Sep 17 00:00:00 2001 From: vinodkc Date: Thu, 13 Nov 2025 13:08:00 -0800 Subject: [PATCH 129/400] [SPARK-54208][CONNECT] Support TIME type in SparkConnectResultSet MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add TIME type support to the Spark Connect JDBC client ### Why are the changes needed? TIME is a fundamental SQL type required for JDBC compliance and interoperability. ### Note : #### After this PR, there is Precision Loss on `SparkConnectResultSet.getTime()` **Overview** When using `ResultSet.getTime()` to retrieve TIME values from Spark Connect JDBC, there is a **precision loss from microseconds to milliseconds**. **The Issue** - **Spark SQL Support**: TIME values with microsecond precision (6 decimal places) - Example: `time '12:34:56.123456'` - **JDBC Limitation**: `java.sql.Time` class only supports millisecond precision (3 decimal places) - **Result**: When calling `getTime()`, the last 3 digits (microseconds) are truncated **Example** ```scala // Input: time '12:34:56.123456' (microseconds: 123456) val time = rs.getTime(1) // Output: time.getTime() = 45296123 (only 123 milliseconds preserved) // 12 hours = 12 × 3,600,000 ms = 43,200,000 ms // 34 minutes = 34 × 60,000 ms = 2,040,000 ms // 56 seconds = 56 × 1,000 ms = 56,000 ms // 123 milliseconds = 123 ms // Total = 45,296,123 ms // Lost: 456 microseconds (the last 3 digits) ``` **Root Cause** This is a fundamental limitation of the `java.sql.Time` class, which internally stores time as milliseconds rather than microseconds. ### Does this PR introduce _any_ user-facing change? Yes, it's part of a new feature under Spark connect JDBC support. ### How was this patch tested? Added new UTs in `SparkConnectJdbcDataTypeSuite` The tests verify that at least the millisecond precision (3 decimal places) is correctly preserved, but users should be aware that any sub-millisecond precision in the original data will be lost when using `getTime()`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53026 from vinodkc/br_SPARK-54208. Authored-by: vinodkc Signed-off-by: Dongjoon Hyun (cherry picked from commit 2a6e6e55cea4e31454abd234d5f9271e2926f59f) Signed-off-by: Dongjoon Hyun --- .../client/jdbc/SparkConnectResultSet.scala | 72 ++++-- .../client/jdbc/util/JdbcTypeUtils.scala | 13 +- .../jdbc/SparkConnectJdbcDataTypeSuite.scala | 205 +++++++++++++----- 3 files changed, 209 insertions(+), 81 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index 5fd425027173a..c4af5b6f12f04 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.connect.client.jdbc import java.io.{InputStream, Reader} import java.net.URL import java.sql.{Array => JdbcArray, _} +import java.time.LocalTime import java.util import java.util.Calendar @@ -147,8 +148,19 @@ class SparkConnectResultSet( getColumnValue(columnIndex, null: Date) { idx => currentRow.getDate(idx) } } - override def getTime(columnIndex: Int): Time = - throw new SQLFeatureNotSupportedException + override def getTime(columnIndex: Int): Time = { + getColumnValue(columnIndex, null: Time) { idx => + val localTime = currentRow.get(idx).asInstanceOf[LocalTime] + // Convert LocalTime to milliseconds since midnight to preserve fractional seconds. + // Note: java.sql.Time can only store up to millisecond precision (3 digits). + // For TIME types with higher precision (TIME(4-9)), microseconds/nanoseconds are truncated. + // If user needs full precision, + // should use: getObject(columnIndex, classOf[java.time.LocalTime]) + val millisSinceMidnight = + java.time.temporal.ChronoUnit.MILLIS.between(LocalTime.MIDNIGHT, localTime) + new Time(millisSinceMidnight) + } + } override def getTimestamp(columnIndex: Int): Timestamp = throw new SQLFeatureNotSupportedException @@ -196,7 +208,7 @@ class SparkConnectResultSet( getDate(findColumn(columnLabel)) override def getTime(columnLabel: String): Time = - throw new SQLFeatureNotSupportedException + getTime(findColumn(columnLabel)) override def getTimestamp(columnLabel: String): Timestamp = throw new SQLFeatureNotSupportedException @@ -500,29 +512,51 @@ class SparkConnectResultSet( override def getArray(columnLabel: String): JdbcArray = throw new SQLFeatureNotSupportedException + /** + * Gets the value of the designated column in the current row as a java.sql.Date object. + * The Calendar parameter is ignored for Date type since it is not timezone-aware. + * + * @param columnIndex the first column is 1, the second is 2, ... + * @param cal the Calendar to use in constructing the date (ignored for Date type) + * @return the column value; if the value is SQL NULL, the value returned is null + */ override def getDate(columnIndex: Int, cal: Calendar): Date = { - val date = getDate(columnIndex) - if (date == null || cal == null) { - return date - } - - val targetCalendar = cal.clone().asInstanceOf[Calendar] - targetCalendar.setTime(date) - targetCalendar.set(Calendar.HOUR_OF_DAY, 0) - targetCalendar.set(Calendar.MINUTE, 0) - targetCalendar.set(Calendar.SECOND, 0) - targetCalendar.set(Calendar.MILLISECOND, 0) - new Date(targetCalendar.getTimeInMillis) + getDate(columnIndex) } + /** + * Gets the value of the designated column in the current row as a java.sql.Date object. + * The Calendar parameter is ignored for Date type since it is not timezone-aware. + * + * @param columnLabel the label for the column specified with the SQL AS clause + * @param cal the Calendar to use in constructing the date (ignored for Date type) + * @return the column value; if the value is SQL NULL, the value returned is null + */ override def getDate(columnLabel: String, cal: Calendar): Date = - getDate(findColumn(columnLabel), cal) + getDate(findColumn(columnLabel)) - override def getTime(columnIndex: Int, cal: Calendar): Time = - throw new SQLFeatureNotSupportedException + /** + * Gets the value of the designated column in the current row as a java.sql.Time object. + * The Calendar parameter is ignored for Time type since it is not timezone-aware. + * + * @param columnIndex the first column is 1, the second is 2, ... + * @param cal the Calendar to use in constructing the time (ignored for Time type) + * @return the column value; if the value is SQL NULL, the value returned is null + */ + override def getTime(columnIndex: Int, cal: Calendar): Time = { + getTime(columnIndex) + } + /** + * Gets the value of the designated column in the current row as a java.sql.Time object. + * The Calendar parameter is ignored for Time type since it is not timezone-aware. + * + * @param columnLabel the label for the column specified with the SQL AS clause + * @param cal the Calendar to use in constructing the time (ignored for Time type) + * @return the column value; if the value is SQL NULL, the value returned is null + */ override def getTime(columnLabel: String, cal: Calendar): Time = - throw new SQLFeatureNotSupportedException + getTime(findColumn(columnLabel)) override def getTimestamp(columnIndex: Int, cal: Calendar): Timestamp = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala index f1cb1236578cc..b3b28359076e2 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala @@ -38,6 +38,7 @@ private[jdbc] object JdbcTypeUtils { case _: DecimalType => Types.DECIMAL case DateType => Types.DATE case BinaryType => Types.BINARY + case _: TimeType => Types.TIME case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -55,6 +56,7 @@ private[jdbc] object JdbcTypeUtils { case _: DecimalType => classOf[JBigDecimal].getName case DateType => classOf[Date].getName case BinaryType => classOf[Array[Byte]].getName + case _: TimeType => classOf[Time].getName case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -62,7 +64,7 @@ private[jdbc] object JdbcTypeUtils { def isSigned(field: StructField): Boolean = field.dataType match { case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | _: DecimalType => true - case NullType | BooleanType | StringType | DateType | BinaryType => false + case NullType | BooleanType | StringType | DateType | BinaryType | _: TimeType => false case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -80,6 +82,11 @@ private[jdbc] object JdbcTypeUtils { case DecimalType.Fixed(p, _) => p case DateType => 10 case BinaryType => Int.MaxValue + // Returns the Spark SQL TIME type precision, even though java.sql.ResultSet.getTime() + // can only retrieve up to millisecond precision (3) due to java.sql.Time limitations. + // Users can call getObject(index, classOf[LocalTime]) to access full microsecond + // precision when the source type is TIME(4) or higher. + case TimeType(precision) => precision case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -88,7 +95,7 @@ private[jdbc] object JdbcTypeUtils { case FloatType => 7 case DoubleType => 15 case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | StringType | - DateType | BinaryType => 0 + DateType | BinaryType | _: TimeType => 0 case DecimalType.Fixed(_, s) => s case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") @@ -105,6 +112,8 @@ private[jdbc] object JdbcTypeUtils { getPrecision(field) case DateType => 10 // length of `YYYY-MM-DD` case BinaryType => Int.MaxValue + case TimeType(precision) if precision > 0 => 8 + 1 + precision // length of `HH:MM:SS.ffffff` + case TimeType(_) => 8 // length of `HH:MM:SS` // precision + negative sign + leading zero + decimal point, like DECIMAL(5,5) = -0.12345 case DecimalType.Fixed(p, s) if p == s => p + 3 // precision + negative sign, like DECIMAL(5,0) = -12345 diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index e06e3474e07b0..370f53be8a2b6 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.connect.client.jdbc import java.sql.{ResultSet, SQLException, Types} +import scala.util.Using + import org.apache.spark.sql.connect.client.jdbc.test.JdbcHelper import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession} @@ -27,6 +29,10 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess override def jdbcUrl: String = s"jdbc:sc://localhost:$serverPort" + private def timeToMillis(hour: Int, minute: Int, second: Int, millis: Int): Long = { + hour * 3600000L + minute * 60000L + second * 1000L + millis + } + test("get null type") { withExecuteQuery("SELECT null") { rs => assert(rs.next()) @@ -260,7 +266,9 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess ("cast(1 AS FLOAT)", (rs: ResultSet) => rs.getFloat(999)), ("cast(1 AS DOUBLE)", (rs: ResultSet) => rs.getDouble(999)), ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(999)), - ("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(999)) + ("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(999)), + ("date '2025-11-15'", (rs: ResultSet) => rs.getBytes(999)), + ("time '12:34:56.123456'", (rs: ResultSet) => rs.getBytes(999)) ).foreach { case (query, getter) => withExecuteQuery(s"SELECT $query") { rs => @@ -287,7 +295,13 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(1), new java.math.BigDecimal("1.00000")), ("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(1), - Array[Byte](0x0A, 0x0B, 0x0C)) + Array[Byte](0x0A, 0x0B, 0x0C)), + ("date '2023-11-15'", (rs: ResultSet) => rs.getDate(1), + java.sql.Date.valueOf("2023-11-15")), + ("time '12:34:56.123456'", (rs: ResultSet) => rs.getTime(1), { + val millis = timeToMillis(12, 34, 56, 123) + new java.sql.Time(millis) + }) ).foreach { case (query, getter, expectedValue) => var resultSet: Option[ResultSet] = None @@ -359,64 +373,6 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess } } - test("get date type with calendar by column index") { - withExecuteQuery("SELECT date '2025-11-15'") { rs => - assert(rs.next()) - - val calUTC = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) - val dateUTC = rs.getDate(1, calUTC) - assert(dateUTC !== null) - assert(!rs.wasNull) - - val calPST = java.util.Calendar.getInstance( - java.util.TimeZone.getTimeZone("America/Los_Angeles")) - val datePST = rs.getDate(1, calPST) - assert(datePST !== null) - assert(!rs.wasNull) - assert(!rs.next()) - } - } - - test("get date type with calendar by column label") { - withExecuteQuery("SELECT date '2025-11-15' as test_date") { rs => - assert(rs.next()) - - val cal = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) - val date = rs.getDate("test_date", cal) - assert(date !== null) - assert(!rs.wasNull) - assert(!rs.next()) - - val metaData = rs.getMetaData - assert(metaData.getColumnCount === 1) - assert(metaData.getColumnName(1) === "test_date") - assert(metaData.getColumnLabel(1) === "test_date") - } - } - - test("get date type with calendar for null value") { - withExecuteQuery("SELECT cast(null as date)") { rs => - assert(rs.next()) - - val cal = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) - val date = rs.getDate(1, cal) - assert(date === null) - assert(rs.wasNull) - assert(!rs.next()) - } - } - - test("get date type with null calendar") { - withExecuteQuery("SELECT date '2025-11-15'") { rs => - assert(rs.next()) - - val date = rs.getDate(1, null) - assert(date === java.sql.Date.valueOf("2025-11-15")) - assert(!rs.wasNull) - assert(!rs.next()) - } - } - test("get binary type") { val testBytes = Array[Byte](0x01, 0x02, 0x03, 0x04, 0x05) val hexString = testBytes.map(b => "%02X".format(b)).mkString @@ -482,4 +438,133 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess assert(!rs.next()) } } + + test("get time type") { + withExecuteQuery("SELECT time '12:34:56.123456'") { rs => + assert(rs.next()) + val time = rs.getTime(1) + // Verify milliseconds are preserved (123 from 123456 microseconds) + val expectedMillis = timeToMillis(12, 34, 56, 123) + assert(time.getTime === expectedMillis) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "TIME '12:34:56.123456'") + assert(metaData.getColumnLabel(1) === "TIME '12:34:56.123456'") + assert(metaData.getColumnType(1) === Types.TIME) + assert(metaData.getColumnTypeName(1) === "TIME(6)") + assert(metaData.getColumnClassName(1) === "java.sql.Time") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 6) + assert(metaData.getScale(1) === 0) + assert(metaData.getColumnDisplaySize(1) === 15) + } + } + + test("get time type with null") { + withExecuteQuery("SELECT cast(null as time)") { rs => + assert(rs.next()) + assert(rs.getTime(1) === null) + assert(rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "CAST(NULL AS TIME(6))") + assert(metaData.getColumnLabel(1) === "CAST(NULL AS TIME(6))") + assert(metaData.getColumnType(1) === Types.TIME) + assert(metaData.getColumnTypeName(1) === "TIME(6)") + assert(metaData.getColumnClassName(1) === "java.sql.Time") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 6) + assert(metaData.getScale(1) === 0) + assert(metaData.getColumnDisplaySize(1) === 15) + } + } + + test("get time type by column label") { + withExecuteQuery("SELECT time '09:15:30.456789' as test_time") { rs => + assert(rs.next()) + val time = rs.getTime("test_time") + // Verify milliseconds are preserved (456 from 456789 microseconds) + val expectedMillis = timeToMillis(9, 15, 30, 456) + assert(time.getTime === expectedMillis) + assert(!rs.wasNull) + assert(!rs.next()) + } + } + + test("get time type with different precisions") { + Seq( + // (timeValue, precision, expectedDisplaySize, expectedMillis) + // HH:MM:SS (no fractional) + ("15:45:30.123456", 0, 8, timeToMillis(15, 45, 30, 0)), + // HH:MM:SS.f (100ms from .1) + ("10:20:30.123456", 1, 10, timeToMillis(10, 20, 30, 100)), + // HH:MM:SS.fff (123ms) + ("08:15:45.123456", 3, 12, timeToMillis(8, 15, 45, 123)), + // HH:MM:SS.fff (999ms) . Spark TIME values can have microsecond precision, + // but java.sql.Time can only store up to millisecond precision + ("23:59:59.999999", 6, 15, timeToMillis(23, 59, 59, 999)) + ).foreach { + case (timeValue, precision, expectedDisplaySize, expectedMillis) => + withExecuteQuery(s"SELECT cast(time '$timeValue' as time($precision))") { rs => + assert(rs.next(), s"Failed to get next row for precision $precision") + val time = rs.getTime(1) + assert(time.getTime === expectedMillis, + s"Time millis mismatch for precision" + + s" $precision: expected $expectedMillis, got ${time.getTime}") + assert(!rs.wasNull, s"wasNull should be false for precision $precision") + assert(!rs.next(), s"Should have no more rows for precision $precision") + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnType(1) === Types.TIME, + s"Column type mismatch for precision $precision") + assert(metaData.getColumnTypeName(1) === s"TIME($precision)", + s"Column type name mismatch for precision $precision") + assert(metaData.getColumnClassName(1) === "java.sql.Time", + s"Column class name mismatch for precision $precision") + assert(metaData.getPrecision(1) === precision, + s"Precision mismatch for precision $precision") + assert(metaData.getScale(1) === 0, + s"Scale should be 0 for precision $precision") + assert(metaData.getColumnDisplaySize(1) === expectedDisplaySize, + s"Display size mismatch for precision $precision: " + + s"expected $expectedDisplaySize, got ${metaData.getColumnDisplaySize(1)}") + } + } + } + + test("get date type with spark.sql.datetime.java8API.enabled") { + withStatement { stmt => + Seq(true, false).foreach { java8APIEnabled => + stmt.execute(s"set spark.sql.datetime.java8API.enabled=$java8APIEnabled") + Using.resource(stmt.executeQuery("SELECT date '2025-11-15'")) { rs => + assert(rs.next()) + assert(rs.getDate(1) === java.sql.Date.valueOf("2025-11-15")) + assert(!rs.wasNull) + assert(!rs.next()) + } + } + } + } + + test("get time type with spark.sql.datetime.java8API.enabled") { + withStatement { stmt => + Seq(true, false).foreach { java8APIEnabled => + stmt.execute(s"set spark.sql.datetime.java8API.enabled=$java8APIEnabled") + Using.resource(stmt.executeQuery("SELECT time '12:34:56.123456'")) { rs => + assert(rs.next()) + val time = rs.getTime(1) + val expectedMillis = timeToMillis(12, 34, 56, 123) + assert(time.getTime === expectedMillis) + assert(!rs.wasNull) + assert(!rs.next()) + } + } + } + } } From 3f506f5232ef4946e60322b685a470c27c21f615 Mon Sep 17 00:00:00 2001 From: Tian Gao Date: Thu, 13 Nov 2025 13:34:06 -0800 Subject: [PATCH 130/400] [SPARK-54340][PYTHON] Add the capability to use viztracer on pyspark daemon/workers ### What changes were proposed in this pull request? Add a script to use viztracer on pyspark daemon/workers without changing the source code. ### Why are the changes needed? Turns out viztracer is very helpful for developers to profile our daemon and workers, including Python UDFs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually tested locally. This should not change the released pyspark at all. It's a dev side tool. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52986 from gaogaotiantian/viztracer. Authored-by: Tian Gao Signed-off-by: Dongjoon Hyun (cherry picked from commit e162b9b0007b8f531dda84b3e1fa1032d0db48ce) Signed-off-by: Dongjoon Hyun --- python/conf_viztracer/daemon_viztracer.py | 48 +++++++++++++++++++ python/conf_viztracer/spark-defaults.conf | 21 ++++++++ python/run-with-viztracer | 58 +++++++++++++++++++++++ 3 files changed, 127 insertions(+) create mode 100644 python/conf_viztracer/daemon_viztracer.py create mode 100644 python/conf_viztracer/spark-defaults.conf create mode 100755 python/run-with-viztracer diff --git a/python/conf_viztracer/daemon_viztracer.py b/python/conf_viztracer/daemon_viztracer.py new file mode 100644 index 0000000000000..2897c10bfe708 --- /dev/null +++ b/python/conf_viztracer/daemon_viztracer.py @@ -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. +# + + +import os +import sys + +import viztracer +from viztracer.main import main + +import pyspark.worker + + +def viztracer_wrapper(func): + + def wrapper(*args, **kwargs): + result = func(*args, **kwargs) + tracer = viztracer.get_tracer() + if tracer is not None: + tracer.exit_routine() + return result + return wrapper + + +if __name__ == "__main__": + pyspark.worker.main = viztracer_wrapper(pyspark.worker.main) + + if os.getenv("SPARK_VIZTRACER_OUTPUT_DIR") is not None: + output_dir = os.getenv("SPARK_VIZTRACER_OUTPUT_DIR") + else: + output_dir = "./" + + sys.argv[:] = ["viztracer", "-m", "pyspark.daemon", "--quiet", "-u", + "--output_dir", output_dir] + main() diff --git a/python/conf_viztracer/spark-defaults.conf b/python/conf_viztracer/spark-defaults.conf new file mode 100644 index 0000000000000..ad84770ff6e0f --- /dev/null +++ b/python/conf_viztracer/spark-defaults.conf @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +spark.python.daemon.module=daemon_viztracer diff --git a/python/run-with-viztracer b/python/run-with-viztracer new file mode 100755 index 0000000000000..448b8e20b9c1d --- /dev/null +++ b/python/run-with-viztracer @@ -0,0 +1,58 @@ +#!/usr/bin/env 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 -o pipefail +set -e + +FWDIR="$(cd "`dirname $0`"; pwd)" + +# Function to display help message +function usage { + cat </dev/null; then + echo "Error: viztracer is not installed. Please install it using 'pip install viztracer'." + exit 1 +fi + +export PYTHONPATH="$FWDIR/conf_viztracer:$PYTHONPATH" +export SPARK_CONF_DIR="$FWDIR/conf_viztracer" +export SPARK_VIZTRACER_OUTPUT_DIR="${SPARK_VIZTRACER_OUTPUT_DIR:-"$(pwd)"}" + +exec "$@" From a54d2aeeea1908ef197bc86b525917a98fd88289 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 13 Nov 2025 14:32:12 -0800 Subject: [PATCH 131/400] [SPARK-54280][SDP] Require pipeline checkpoint storage dir to be absolute path ### What changes were proposed in this pull request? - Raises an error if the pipeline checkpoint storage dir is not an absolute path - Updated the init CLI to create and set a checkpoint storage dir as an absolute path ### Why are the changes needed? Prevent users from accidentally losing checkpoints. ### Does this PR introduce _any_ user-facing change? Yes, but to unreleased functionality. ### How was this patch tested? - New unit tests - Ran the init CLI and then ran pipeline with streaming table ### Was this patch authored or co-authored using generative AI tooling? Closes #52999 from sryza/storage-location-absolute. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 4020794c719e63bd82abd67e63f699a28e4ce9f5) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 6 ++ python/pyspark/pipelines/init_cli.py | 12 ++- .../pyspark/pipelines/tests/test_init_cli.py | 8 ++ .../SparkConnectSessionHolderSuite.scala | 2 +- .../SparkConnectSessionManagerSuite.scala | 2 +- .../graph/PipelineUpdateContextImpl.scala | 21 ++++ .../PipelineUpdateContextImplSuite.scala | 98 +++++++++++++++++++ .../pipelines/graph/SinkExecutionSuite.scala | 2 +- .../pipelines/utils/StorageRootMixin.scala | 2 +- 9 files changed, 147 insertions(+), 6 deletions(-) create mode 100644 sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImplSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 489f3fbad34ec..9fe18fac66fef 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4997,6 +4997,12 @@ ], "sqlState" : "42000" }, + "PIPELINE_STORAGE_ROOT_INVALID" : { + "message" : [ + "Pipeline storage root must be an absolute path with a URI scheme (e.g., file://, s3a://, hdfs://). Got: ``." + ], + "sqlState" : "42K03" + }, "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION" : { "message" : [ "Non-grouping expression is provided as an argument to the |> AGGREGATE pipe operator but does not contain any aggregate function; please update it to include an aggregate function and then retry the query again." diff --git a/python/pyspark/pipelines/init_cli.py b/python/pyspark/pipelines/init_cli.py index ffe5d3c12b636..f8149b19263f8 100644 --- a/python/pyspark/pipelines/init_cli.py +++ b/python/pyspark/pipelines/init_cli.py @@ -19,7 +19,7 @@ SPEC = """ name: {{ name }} -storage: storage-root +storage: {{ storage_root }} libraries: - glob: include: transformations/** @@ -46,10 +46,18 @@ def init(name: str) -> None: project_dir = Path.cwd() / name project_dir.mkdir(parents=True, exist_ok=False) + # Create the storage directory + storage_dir = project_dir / "pipeline-storage" + storage_dir.mkdir(parents=True) + + # Create absolute file URI for storage path + storage_path = f"file://{storage_dir.resolve()}" + # Write the spec file to the project directory spec_file = project_dir / "pipeline.yml" with open(spec_file, "w") as f: - f.write(SPEC.replace("{{ name }}", name)) + spec_content = SPEC.replace("{{ name }}", name).replace("{{ storage_root }}", storage_path) + f.write(spec_content) # Create the transformations directory transformations_dir = project_dir / "transformations" diff --git a/python/pyspark/pipelines/tests/test_init_cli.py b/python/pyspark/pipelines/tests/test_init_cli.py index 49c949200821a..43c553eddc387 100644 --- a/python/pyspark/pipelines/tests/test_init_cli.py +++ b/python/pyspark/pipelines/tests/test_init_cli.py @@ -51,6 +51,14 @@ def test_init(self): spec_path = find_pipeline_spec(Path.cwd()) spec = load_pipeline_spec(spec_path) assert spec.name == project_name + + # Verify that the storage path is an absolute URI with file scheme + expected_storage_path = f"file://{Path.cwd() / 'pipeline-storage'}" + self.assertEqual(spec.storage, expected_storage_path) + + # Verify that the storage directory was created + self.assertTrue((Path.cwd() / "pipeline-storage").exists()) + registry = LocalGraphElementRegistry() register_definitions(spec_path, registry, spec) self.assertEqual(len(registry.outputs), 1) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala index 680755afdca21..1b747705e9ad7 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala @@ -437,7 +437,7 @@ class SparkConnectSessionHolderSuite extends SharedSparkSession { val pipelineUpdateContext = new PipelineUpdateContextImpl( new DataflowGraph(Seq(), Seq(), Seq(), Seq()), (_: PipelineEvent) => None, - storageRoot = "test_storage_root") + storageRoot = "file:///test_storage_root") sessionHolder.cachePipelineExecution(graphId, pipelineUpdateContext) assert( sessionHolder.getPipelineExecution(graphId).nonEmpty, diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala index a3d851c1ce7b9..94deb83f6ad43 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala @@ -161,7 +161,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val pipelineUpdateContext = new PipelineUpdateContextImpl( new DataflowGraph(Seq(), Seq(), Seq(), Seq()), (_: PipelineEvent) => None, - storageRoot = "test_storage_root") + storageRoot = "file:///test_storage_root") sessionHolder.cachePipelineExecution(graphId, pipelineUpdateContext) assert( sessionHolder.getPipelineExecution(graphId).nonEmpty, diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImpl.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImpl.scala index bb2009b259124..7dfffbd9d662c 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImpl.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImpl.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.pipelines.graph +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.pipelines.logging.{FlowProgressEventLogger, PipelineEvent} @@ -36,6 +39,8 @@ class PipelineUpdateContextImpl( override val storageRoot: String ) extends PipelineUpdateContext { + PipelineUpdateContextImpl.validateStorageRoot(storageRoot) + override val spark: SparkSession = SparkSession.getActiveSession.getOrElse( throw new IllegalStateException("SparkSession is not available") ) @@ -45,3 +50,19 @@ class PipelineUpdateContextImpl( override val resetCheckpointFlows: FlowFilter = NoFlows } + +object PipelineUpdateContextImpl { + def validateStorageRoot(storageRoot: String): Unit = { + // Use the same validation logic as streaming checkpoint directories + val path = new Path(storageRoot) + + val uri = path.toUri + if (!path.isAbsolute || uri.getScheme == null || uri.getScheme.isEmpty) { + throw new SparkException( + errorClass = "PIPELINE_STORAGE_ROOT_INVALID", + messageParameters = Map("storage_root" -> storageRoot), + cause = null + ) + } + } +} diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImplSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImplSuite.scala new file mode 100644 index 0000000000000..b22b6a9967e04 --- /dev/null +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/PipelineUpdateContextImplSuite.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.pipelines.graph + +import org.apache.spark.SparkException +import org.apache.spark.sql.pipelines.utils.{PipelineTest, TestGraphRegistrationContext} +import org.apache.spark.sql.test.SharedSparkSession + +class PipelineUpdateContextImplSuite extends PipelineTest with SharedSparkSession { + + test("validateStorageRoot should accept valid URIs with schemes") { + val validStorageRoots = Seq( + "file:///tmp/test", + "hdfs://localhost:9000/pipelines", + "s3a://my-bucket/pipelines", + "abfss://container@account.dfs.core.windows.net/pipelines" + ) + + validStorageRoots.foreach(PipelineUpdateContextImpl.validateStorageRoot) + } + + test("validateStorageRoot should reject relative paths") { + val invalidStorageRoots = Seq( + "relative/path", + "./relative/path", + "../relative/path", + "pipelines" + ) + + invalidStorageRoots.foreach { storageRoot => + val exception = intercept[SparkException] { + PipelineUpdateContextImpl.validateStorageRoot(storageRoot) + } + assert(exception.getCondition == "PIPELINE_STORAGE_ROOT_INVALID") + assert(exception.getMessageParameters.get("storage_root") == storageRoot) + } + } + + test("validateStorageRoot should reject absolute paths without URI scheme") { + val invalidStorageRoots = Seq( + "/tmp/test", + "/absolute/path", + "/pipelines/storage" + ) + + invalidStorageRoots.foreach { storageRoot => + val exception = intercept[SparkException] { + PipelineUpdateContextImpl.validateStorageRoot(storageRoot) + } + assert(exception.getCondition == "PIPELINE_STORAGE_ROOT_INVALID") + assert(exception.getMessageParameters.get("storage_root") == storageRoot) + } + } + + test("PipelineUpdateContextImpl constructor should validate storage root") { + val session = spark + import session.implicits._ + + class TestPipeline extends TestGraphRegistrationContext(spark) { + registerPersistedView("test", query = dfFlowFunc(Seq(1).toDF("value"))) + } + val graph = new TestPipeline().resolveToDataflowGraph() + + val validStorageRoot = "file:///tmp/test" + val context = new PipelineUpdateContextImpl( + unresolvedGraph = graph, + eventCallback = _ => {}, + storageRoot = validStorageRoot + ) + assert(context.storageRoot == validStorageRoot) + + val invalidStorageRoot = "/tmp/test" + val exception = intercept[SparkException] { + new PipelineUpdateContextImpl( + unresolvedGraph = graph, + eventCallback = _ => {}, + storageRoot = invalidStorageRoot + ) + } + assert(exception.getCondition == "PIPELINE_STORAGE_ROOT_INVALID") + assert(exception.getMessageParameters.get("storage_root") == invalidStorageRoot) + } +} diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SinkExecutionSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SinkExecutionSuite.scala index 958ef5a80fd57..9e6010a611e97 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SinkExecutionSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SinkExecutionSuite.scala @@ -115,7 +115,7 @@ class SinkExecutionSuite extends ExecutionTest with SharedSparkSession { sinkIdentifier: TableIdentifier, flowIdentifier: TableIdentifier): Unit = { val expectedCheckpointLocation = new Path( - "file://" + rootDirectory + s"/_checkpoints/${sinkIdentifier.table}/${flowIdentifier.table}/0" + rootDirectory + s"/_checkpoints/${sinkIdentifier.table}/${flowIdentifier.table}/0" ) val streamingQuery = graphExecution .flowExecutions(flowIdentifier) diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/StorageRootMixin.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/StorageRootMixin.scala index 420e2c6ad0e91..7c998c6df3d53 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/StorageRootMixin.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/StorageRootMixin.scala @@ -38,7 +38,7 @@ trait StorageRootMixin extends BeforeAndAfterEach { self: Suite => override protected def beforeEach(): Unit = { super.beforeEach() storageRoot = - Files.createTempDirectory(getClass.getSimpleName).normalize.toString + s"file://${Files.createTempDirectory(getClass.getSimpleName).normalize.toString}" } override protected def afterEach(): Unit = { From aefde261c4874d39e65cb141366381ddefcf10db Mon Sep 17 00:00:00 2001 From: vinodkc Date: Thu, 13 Nov 2025 15:06:18 -0800 Subject: [PATCH 132/400] [SPARK-54209][CONNECT] Support TIMESTAMP type in SparkConnectResultSet ### What changes were proposed in this pull request? Add TIMESTAMP type support to the Spark Connect JDBC client ### Why are the changes needed? TIMESTAMP is a fundamental SQL type required for JDBC compliance and interoperability. ### Does this PR introduce _any_ user-facing change? Yes, it's part of a new feature under Spark connect JDBC support. ### How was this patch tested? Added new UTs in `SparkConnectJdbcDataTypeSuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #53046 from vinodkc/br_SPARK-54209. Authored-by: vinodkc Signed-off-by: Dongjoon Hyun (cherry picked from commit 9f1dc1ccf8b7af08d056f8f30c8ef0b091550657) Signed-off-by: Dongjoon Hyun --- .../client/jdbc/SparkConnectResultSet.scala | 41 +++- .../client/jdbc/util/JdbcTypeUtils.scala | 13 +- .../jdbc/SparkConnectJdbcDataTypeSuite.scala | 182 +++++++++++++++++- .../jdbc/SparkConnectResultSetSuite.scala | 64 ++++++ 4 files changed, 292 insertions(+), 8 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index c4af5b6f12f04..8b2d4578d0aa6 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -27,6 +27,7 @@ import java.util.Calendar import org.apache.spark.sql.Row import org.apache.spark.sql.connect.client.SparkResult import org.apache.spark.sql.connect.client.jdbc.util.JdbcErrorUtils +import org.apache.spark.sql.types.{TimestampNTZType, TimestampType} class SparkConnectResultSet( sparkResult: SparkResult[Row], @@ -162,8 +163,27 @@ class SparkConnectResultSet( } } - override def getTimestamp(columnIndex: Int): Timestamp = - throw new SQLFeatureNotSupportedException + override def getTimestamp(columnIndex: Int): Timestamp = { + getColumnValue(columnIndex, null: Timestamp) { idx => + val value = currentRow.get(idx) + if (value == null) { + null + } else { + sparkResult.schema.fields(idx).dataType match { + case TimestampNTZType => + // TIMESTAMP_NTZ is represented as LocalDateTime + Timestamp.valueOf(value.asInstanceOf[java.time.LocalDateTime]) + case TimestampType => + // TIMESTAMP is represented as java.sql.Timestamp + value.asInstanceOf[Timestamp] + case other => + throw new SQLException( + s"Cannot call getTimestamp() on column of type $other. " + + s"Expected TIMESTAMP or TIMESTAMP_NTZ.") + } + } + } + } override def getAsciiStream(columnIndex: Int): InputStream = throw new SQLFeatureNotSupportedException @@ -211,7 +231,7 @@ class SparkConnectResultSet( getTime(findColumn(columnLabel)) override def getTimestamp(columnLabel: String): Timestamp = - throw new SQLFeatureNotSupportedException + getTimestamp(findColumn(columnLabel)) override def getAsciiStream(columnLabel: String): InputStream = throw new SQLFeatureNotSupportedException @@ -558,11 +578,20 @@ class SparkConnectResultSet( override def getTime(columnLabel: String, cal: Calendar): Time = getTime(findColumn(columnLabel)) - override def getTimestamp(columnIndex: Int, cal: Calendar): Timestamp = - throw new SQLFeatureNotSupportedException + /** + * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions + * server-side to avoid client/server timezone inconsistencies. + */ + override def getTimestamp(columnIndex: Int, cal: Calendar): Timestamp = { + getTimestamp(columnIndex) + } + /** + * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions + * server-side to avoid client/server timezone inconsistencies. + */ override def getTimestamp(columnLabel: String, cal: Calendar): Timestamp = - throw new SQLFeatureNotSupportedException + getTimestamp(findColumn(columnLabel), cal) override def getURL(columnIndex: Int): URL = throw new SQLFeatureNotSupportedException diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala index b3b28359076e2..91de849e46c03 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala @@ -37,6 +37,8 @@ private[jdbc] object JdbcTypeUtils { case StringType => Types.VARCHAR case _: DecimalType => Types.DECIMAL case DateType => Types.DATE + case TimestampType => Types.TIMESTAMP + case TimestampNTZType => Types.TIMESTAMP case BinaryType => Types.BINARY case _: TimeType => Types.TIME case other => @@ -55,6 +57,8 @@ private[jdbc] object JdbcTypeUtils { case StringType => classOf[String].getName case _: DecimalType => classOf[JBigDecimal].getName case DateType => classOf[Date].getName + case TimestampType => classOf[Timestamp].getName + case TimestampNTZType => classOf[Timestamp].getName case BinaryType => classOf[Array[Byte]].getName case _: TimeType => classOf[Time].getName case other => @@ -64,7 +68,8 @@ private[jdbc] object JdbcTypeUtils { def isSigned(field: StructField): Boolean = field.dataType match { case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | _: DecimalType => true - case NullType | BooleanType | StringType | DateType | BinaryType | _: TimeType => false + case NullType | BooleanType | StringType | DateType | BinaryType | _: TimeType | + TimestampType | TimestampNTZType => false case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } @@ -81,6 +86,8 @@ private[jdbc] object JdbcTypeUtils { case StringType => 255 case DecimalType.Fixed(p, _) => p case DateType => 10 + case TimestampType => 29 + case TimestampNTZType => 29 case BinaryType => Int.MaxValue // Returns the Spark SQL TIME type precision, even though java.sql.ResultSet.getTime() // can only retrieve up to millisecond precision (3) due to java.sql.Time limitations. @@ -94,6 +101,8 @@ private[jdbc] object JdbcTypeUtils { def getScale(field: StructField): Int = field.dataType match { case FloatType => 7 case DoubleType => 15 + case TimestampType => 6 + case TimestampNTZType => 6 case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | StringType | DateType | BinaryType | _: TimeType => 0 case DecimalType.Fixed(_, s) => s @@ -111,6 +120,8 @@ private[jdbc] object JdbcTypeUtils { case StringType => getPrecision(field) case DateType => 10 // length of `YYYY-MM-DD` + case TimestampType => 29 // length of `YYYY-MM-DD HH:MM:SS.SSSSSS` + case TimestampNTZType => 29 // length of `YYYY-MM-DD HH:MM:SS.SSSSSS` case BinaryType => Int.MaxValue case TimeType(precision) if precision > 0 => 8 + 1 + precision // length of `HH:MM:SS.ffffff` case TimeType(_) => 8 // length of `HH:MM:SS` diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index 370f53be8a2b6..79b5eb8a052bf 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -268,7 +268,9 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess ("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(999)), ("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(999)), ("date '2025-11-15'", (rs: ResultSet) => rs.getBytes(999)), - ("time '12:34:56.123456'", (rs: ResultSet) => rs.getBytes(999)) + ("time '12:34:56.123456'", (rs: ResultSet) => rs.getBytes(999)), + ("timestamp '2025-11-15 10:30:45.123456'", (rs: ResultSet) => rs.getTimestamp(999)), + ("timestamp_ntz '2025-11-15 10:30:45.789012'", (rs: ResultSet) => rs.getTimestamp(999)), ).foreach { case (query, getter) => withExecuteQuery(s"SELECT $query") { rs => @@ -567,4 +569,182 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess } } } + +test("get timestamp type") { + withExecuteQuery("SELECT timestamp '2025-11-15 10:30:45.123456'") { rs => + assert(rs.next()) + val timestamp = rs.getTimestamp(1) + assert(timestamp !== null) + assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.123456")) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "TIMESTAMP '2025-11-15 10:30:45.123456'") + assert(metaData.getColumnLabel(1) === "TIMESTAMP '2025-11-15 10:30:45.123456'") + assert(metaData.getColumnType(1) === Types.TIMESTAMP) + assert(metaData.getColumnTypeName(1) === "TIMESTAMP") + assert(metaData.getColumnClassName(1) === "java.sql.Timestamp") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 29) + assert(metaData.getScale(1) === 6) + assert(metaData.getColumnDisplaySize(1) === 29) + } + } + + test("get timestamp type with null") { + withExecuteQuery("SELECT cast(null as timestamp)") { rs => + assert(rs.next()) + assert(rs.getTimestamp(1) === null) + assert(rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "CAST(NULL AS TIMESTAMP)") + assert(metaData.getColumnLabel(1) === "CAST(NULL AS TIMESTAMP)") + assert(metaData.getColumnType(1) === Types.TIMESTAMP) + assert(metaData.getColumnTypeName(1) === "TIMESTAMP") + assert(metaData.getColumnClassName(1) === "java.sql.Timestamp") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 29) + assert(metaData.getScale(1) === 6) + assert(metaData.getColumnDisplaySize(1) === 29) + } + } + + test("get timestamp type by column label and with calendar") { + withExecuteQuery("SELECT timestamp '2025-11-15 10:30:45.987654' as test_timestamp") { rs => + assert(rs.next()) + + // Test by column label + val timestamp = rs.getTimestamp("test_timestamp") + assert(timestamp !== null) + assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.987654")) + assert(!rs.wasNull) + + // Test with calendar - should return same value (Calendar is ignored) + // Note: Spark Connect handles timezone at server, Calendar param is for API compliance + val calUTC = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) + val timestampUTC = rs.getTimestamp(1, calUTC) + assert(timestampUTC !== null) + assert(timestampUTC.getTime === timestamp.getTime) + + val calPST = java.util.Calendar.getInstance( + java.util.TimeZone.getTimeZone("America/Los_Angeles")) + val timestampPST = rs.getTimestamp(1, calPST) + assert(timestampPST !== null) + // Same value regardless of calendar + assert(timestampPST.getTime === timestamp.getTime) + assert(timestampUTC.getTime === timestampPST.getTime) + + // Test with calendar by label + val timestampLabel = rs.getTimestamp("test_timestamp", calUTC) + assert(timestampLabel !== null) + assert(timestampLabel.getTime === timestamp.getTime) + + // Test with null calendar - returns same value + val timestampNullCal = rs.getTimestamp(1, null) + assert(timestampNullCal !== null) + assert(timestampNullCal.getTime === timestamp.getTime) + + assert(!rs.next()) + } + } + + test("get timestamp type with calendar for null value") { + withExecuteQuery("SELECT cast(null as timestamp)") { rs => + assert(rs.next()) + + // Calendar parameter should not affect null handling + val cal = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) + val timestamp = rs.getTimestamp(1, cal) + assert(timestamp === null) + assert(rs.wasNull) + assert(!rs.next()) + } + } + + test("get timestamp_ntz type") { + withExecuteQuery("SELECT timestamp_ntz '2025-11-15 10:30:45.123456'") { rs => + assert(rs.next()) + val timestamp = rs.getTimestamp(1) + assert(timestamp !== null) + assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.123456")) + assert(!rs.wasNull) + assert(!rs.next()) + + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 1) + assert(metaData.getColumnName(1) === "TIMESTAMP_NTZ '2025-11-15 10:30:45.123456'") + assert(metaData.getColumnLabel(1) === "TIMESTAMP_NTZ '2025-11-15 10:30:45.123456'") + assert(metaData.getColumnType(1) === Types.TIMESTAMP) + assert(metaData.getColumnTypeName(1) === "TIMESTAMP_NTZ") + assert(metaData.getColumnClassName(1) === "java.sql.Timestamp") + assert(metaData.isSigned(1) === false) + assert(metaData.getPrecision(1) === 29) + assert(metaData.getScale(1) === 6) + assert(metaData.getColumnDisplaySize(1) === 29) + } + } + + test("get timestamp_ntz type by label, null, and with calendar") { + // Test with non-null value + withExecuteQuery("SELECT timestamp_ntz '2025-11-15 14:22:33.789456' as test_ts_ntz") { rs => + assert(rs.next()) + + // Test by column label + val timestamp = rs.getTimestamp("test_ts_ntz") + assert(timestamp !== null) + assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 14:22:33.789456")) + assert(!rs.wasNull) + + // Test with calendar - should return same value (Calendar is ignored) + val calUTC = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC")) + val timestampCal = rs.getTimestamp(1, calUTC) + assert(timestampCal !== null) + assert(timestampCal.getTime === timestamp.getTime) + + assert(!rs.next()) + } + + // Test with null value + withExecuteQuery("SELECT cast(null as timestamp_ntz)") { rs => + assert(rs.next()) + assert(rs.getTimestamp(1) === null) + assert(rs.wasNull) + assert(!rs.next()) + } + } + + test("get timestamp types with spark.sql.datetime.java8API.enabled") { + withStatement { stmt => + Seq(true, false).foreach { java8APIEnabled => + stmt.execute(s"set spark.sql.datetime.java8API.enabled=$java8APIEnabled") + + Using.resource(stmt.executeQuery( + """SELECT + | timestamp '2025-11-15 10:30:45.123456' as ts, + | timestamp_ntz '2025-11-15 14:22:33.789012' as ts_ntz + |""".stripMargin)) { rs => + assert(rs.next()) + + // Test TIMESTAMP type + val timestamp = rs.getTimestamp(1) + assert(timestamp !== null) + assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.123456")) + assert(!rs.wasNull) + + // Test TIMESTAMP_NTZ type + val timestampNtz = rs.getTimestamp(2) + assert(timestampNtz !== null) + assert(timestampNtz === java.sql.Timestamp.valueOf("2025-11-15 14:22:33.789012")) + assert(!rs.wasNull) + + assert(!rs.next()) + } + } + } + } } diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSetSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSetSuite.scala index ac2866837e939..21b8e261aef45 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSetSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSetSuite.scala @@ -122,4 +122,68 @@ class SparkConnectResultSetSuite extends ConnectFunSuite with RemoteSparkSession assert(rs.isAfterLast) } } + + test("getTimestamp with multiple columns, rows, and types") { + withExecuteQuery( + """SELECT ts_tz, ts_ntz, id FROM VALUES + | (timestamp '2025-01-15 10:30:45.123456', timestamp_ntz '2025-06-20 14:22:33.789012', 1), + | (null, timestamp_ntz '2025-03-01 18:30:45.456789', 2), + | (timestamp '2025-10-31 23:59:59.999999', null, 3) + | AS t(ts_tz, ts_ntz, id) + |""".stripMargin) { rs => + + // Test findColumn + assert(rs.findColumn("ts_tz") === 1) + assert(rs.findColumn("ts_ntz") === 2) + assert(rs.findColumn("id") === 3) + + // Verify metadata + val metaData = rs.getMetaData + assert(metaData.getColumnCount === 3) + assert(metaData.getColumnTypeName(1) === "TIMESTAMP") + assert(metaData.getColumnTypeName(2) === "TIMESTAMP_NTZ") + + // Row 1: Both timestamps have values + assert(rs.next()) + assert(rs.getRow === 1) + + val ts1 = rs.getTimestamp(1) + assert(ts1 !== null) + assert(ts1 === java.sql.Timestamp.valueOf("2025-01-15 10:30:45.123456")) + assert(!rs.wasNull) + + val tsNtz1 = rs.getTimestamp("ts_ntz") + assert(tsNtz1 !== null) + assert(tsNtz1 === java.sql.Timestamp.valueOf("2025-06-20 14:22:33.789012")) + assert(!rs.wasNull) + + // Row 2: TIMESTAMP is null, TIMESTAMP_NTZ has value + assert(rs.next()) + assert(rs.getRow === 2) + + val ts2 = rs.getTimestamp(1) + assert(ts2 === null) + assert(rs.wasNull) + + val tsNtz2 = rs.getTimestamp(2) + assert(tsNtz2 !== null) + assert(tsNtz2 === java.sql.Timestamp.valueOf("2025-03-01 18:30:45.456789")) + assert(!rs.wasNull) + + // Row 3: TIMESTAMP has value, TIMESTAMP_NTZ is null + assert(rs.next()) + assert(rs.getRow === 3) + + val ts3 = rs.getTimestamp("ts_tz") + assert(ts3 !== null) + assert(ts3 === java.sql.Timestamp.valueOf("2025-10-31 23:59:59.999999")) + assert(!rs.wasNull) + + val tsNtz3 = rs.getTimestamp(2) + assert(tsNtz3 === null) + assert(rs.wasNull) + + assert(!rs.next()) + } + } } From 2e846d282c66f9e7b92478f7eb4b1b0b767bc961 Mon Sep 17 00:00:00 2001 From: vinodkc Date: Thu, 13 Nov 2025 16:18:53 -0800 Subject: [PATCH 133/400] [SPARK-54209][CONNECT] [Followup] Fix linter error in `SparkConnectJdbcDataTypeSuite` ### What changes were proposed in this pull request? Followup of https://github.com/apache/spark/pull/53046 to fix compiler error ### Why are the changes needed? To fix build failure ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Built locally and ran `./dev/lint-scala` ### Was this patch authored or co-authored using generative AI tooling? No Closes #53049 from vinodkc/br_fix_lint_SPARK-54209. Authored-by: vinodkc Signed-off-by: Dongjoon Hyun (cherry picked from commit 87b3b94232436528f88c9a7aa7ee70758b85a33a) Signed-off-by: Dongjoon Hyun --- .../connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala index 79b5eb8a052bf..3a02f78c43831 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectJdbcDataTypeSuite.scala @@ -251,7 +251,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess assert(metaData.getScale(1) === scale) assert(metaData.getColumnDisplaySize(1) === expectedColumnDisplaySize) assert(metaData.getColumnDisplaySize(1) >= value.size) - } + } } } @@ -270,7 +270,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess ("date '2025-11-15'", (rs: ResultSet) => rs.getBytes(999)), ("time '12:34:56.123456'", (rs: ResultSet) => rs.getBytes(999)), ("timestamp '2025-11-15 10:30:45.123456'", (rs: ResultSet) => rs.getTimestamp(999)), - ("timestamp_ntz '2025-11-15 10:30:45.789012'", (rs: ResultSet) => rs.getTimestamp(999)), + ("timestamp_ntz '2025-11-15 10:30:45.789012'", (rs: ResultSet) => rs.getTimestamp(999)) ).foreach { case (query, getter) => withExecuteQuery(s"SELECT $query") { rs => @@ -570,7 +570,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess } } -test("get timestamp type") { + test("get timestamp type") { withExecuteQuery("SELECT timestamp '2025-11-15 10:30:45.123456'") { rs => assert(rs.next()) val timestamp = rs.getTimestamp(1) From 0dc8135eb5cfa8ce1a2455e08bcd36fcec1d986a Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 13 Nov 2025 17:48:58 -0800 Subject: [PATCH 134/400] [SPARK-54341][SQL] Remember TimeTravelSpec for tables loaded via TableProvider ### What changes were proposed in this pull request? This PR adds logic to remember `TimeTravelSpec` for tables loaded via `TableProvider`. ### Why are the changes needed? These changes are needed to align `TableProvider` with other places that remember time travel specs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR comes with tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53048 from aokolnychyi/spark-54341. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun (cherry picked from commit 116240c39507a3ef46d3be1ad329520291b70096) Signed-off-by: Dongjoon Hyun --- .../datasources/v2/DataSourceV2Utils.scala | 9 ++-- .../SupportsCatalogOptionsSuite.scala | 44 ++++++++++++++----- 2 files changed, 37 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index 92bbca3c02966..a3b5c5aeb7995 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -115,7 +115,7 @@ private[sql] object DataSourceV2Utils extends Logging { val finalOptions = sessionOptions.filter { case (k, _) => !optionsWithPath.contains(k) } ++ optionsWithPath.originalMap val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) - val (table, catalog, ident) = provider match { + val (table, catalog, ident, timeTravelSpec) = provider match { case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty => throw new IllegalArgumentException( s"$source does not support user specified schema. Please don't specify the schema.") @@ -141,16 +141,17 @@ private[sql] object DataSourceV2Utils extends Logging { } val timeTravel = TimeTravelSpec.create( timeTravelTimestamp, timeTravelVersion, conf.sessionLocalTimeZone) - (CatalogV2Util.getTable(catalog, ident, timeTravel), Some(catalog), Some(ident)) + val tbl = CatalogV2Util.getTable(catalog, ident, timeTravel) + (tbl, Some(catalog), Some(ident), timeTravel) case _ => // TODO: Non-catalog paths for DSV2 are currently not well defined. val tbl = DataSourceV2Utils.getTableFromProvider(provider, dsOptions, userSpecifiedSchema) - (tbl, None, None) + (tbl, None, None, None) } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case _: SupportsRead if table.supports(BATCH_READ) => - Option(DataSourceV2Relation.create(table, catalog, ident, dsOptions)) + Option(DataSourceV2Relation.create(table, catalog, ident, dsOptions, timeTravelSpec)) case _ => None } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 93a8f70c57982..6b5bd982ee5a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -26,8 +26,8 @@ import scala.util.Try import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, SaveMode} -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, SaveMode} +import org.apache.spark.sql.catalyst.analysis.{AsOfTimestamp, AsOfVersion, NoSuchTableException, TableAlreadyExistsException, TimeTravelSpec} import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, InMemoryTableCatalog, SupportsCatalogOptions, TableCatalog} @@ -305,8 +305,13 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with .mode(SaveMode.Overwrite).save() // load with version - checkAnswer(load("t", Some(catalogName), version = Some("Snapshot123456789")), df1.toDF()) - checkAnswer(load("t", Some(catalogName), version = Some("2345678910")), df2.toDF()) + val readDF1 = load("t", Some(catalogName), version = Some("Snapshot123456789")) + checkAnswer(readDF1, df1.toDF()) + checkTimeTravel(readDF1, expectedTimeTravelSpec = AsOfVersion("Snapshot123456789")) + + val readDF2 = load("t", Some(catalogName), version = Some("2345678910")) + checkAnswer(readDF2, df2.toDF()) + checkTimeTravel(readDF2, expectedTimeTravelSpec = AsOfVersion("2345678910")) } val ts1 = DateTimeUtils.stringToTimestampAnsi( @@ -330,16 +335,26 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with .mode(SaveMode.Overwrite).save() // load with timestamp - checkAnswer(load("t", Some(catalogName), version = None, - timestamp = Some("2019-01-29 00:37:58")), df3.toDF()) - checkAnswer(load("t", Some(catalogName), version = None, - timestamp = Some("2021-01-29 00:37:58")), df4.toDF()) + val readDF3 = load("t", Some(catalogName), version = None, + timestamp = Some("2019-01-29 00:37:58")) + checkAnswer(readDF3, df3.toDF()) + checkTimeTravel(readDF3, expectedTimeTravelSpec = AsOfTimestamp(ts1)) + + val readDF4 = load("t", Some(catalogName), version = None, + timestamp = Some("2021-01-29 00:37:58")) + checkAnswer(readDF4, df4.toDF()) + checkTimeTravel(readDF4, expectedTimeTravelSpec = AsOfTimestamp(ts2)) // load with timestamp in number format - checkAnswer(load("t", Some(catalogName), version = None, - timestamp = Some(MICROSECONDS.toSeconds(ts1).toString)), df3.toDF()) - checkAnswer(load("t", Some(catalogName), version = None, - timestamp = Some(MICROSECONDS.toSeconds(ts2).toString)), df4.toDF()) + val readDF5 = load("t", Some(catalogName), version = None, + timestamp = Some(MICROSECONDS.toSeconds(ts1).toString)) + checkAnswer(readDF5, df3.toDF()) + checkTimeTravel(readDF5, expectedTimeTravelSpec = AsOfTimestamp(ts1)) + + val readDF6 = load("t", Some(catalogName), version = None, + timestamp = Some(MICROSECONDS.toSeconds(ts2).toString)) + checkAnswer(readDF6, df4.toDF()) + checkTimeTravel(readDF6, expectedTimeTravelSpec = AsOfTimestamp(ts2)) } val e = intercept[AnalysisException] { @@ -360,6 +375,11 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with assert(v2.catalog.exists(_ == catalogPlugin)) } + private def checkTimeTravel(ds: Dataset[_], expectedTimeTravelSpec: TimeTravelSpec): Unit = { + val relation = ds.logicalPlan.asInstanceOf[DataSourceV2Relation] + assert(relation.timeTravelSpec.contains(expectedTimeTravelSpec)) + } + private def load( name: String, catalogOpt: Option[String], From 0ca1bad95509cc0cad2872fb088c7a41f0d6806a Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 13 Nov 2025 19:44:17 -0800 Subject: [PATCH 135/400] [SPARK-54332][PYTHON][CONNECT] No need to attach PlanId in grouping column names in rollup/cube/groupingSets ### What changes were proposed in this pull request? A follow up of https://github.com/apache/spark/pull/52933, avoiding attaching plan id of column names in rollup/cube/groupingSets ### Why are the changes needed? to be consistent with classic: https://github.com/apache/spark/blob/e75ca577923f9f465eb06b4df814c00143fa41ea/sql/api/src/main/scala/org/apache/spark/sql/Dataset.scala#L1415-L1416 https://github.com/apache/spark/blob/e75ca577923f9f465eb06b4df814c00143fa41ea/sql/api/src/main/scala/org/apache/spark/sql/Dataset.scala#L1366-L1368 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53033 from zhengruifeng/no_plan_id_cube. Authored-by: Ruifeng Zheng Signed-off-by: Wenchen Fan (cherry picked from commit 9f1bd47bab158e7743db1c1798612125eef01003) Signed-off-by: Wenchen Fan --- python/pyspark/sql/connect/dataframe.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 862974f111658..6a448025932b6 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -619,7 +619,7 @@ def rollup(self, *cols: "ColumnOrNameOrOrdinal") -> "GroupedData": # type: igno if isinstance(c, Column): _cols.append(c) elif isinstance(c, str): - _cols.append(self[c]) + _cols.append(F.col(c)) elif isinstance(c, int) and not isinstance(c, bool): if c < 1: raise PySparkIndexError( @@ -649,7 +649,7 @@ def cube(self, *cols: "ColumnOrName") -> "GroupedData": # type: ignore[misc] if isinstance(c, Column): _cols.append(c) elif isinstance(c, str): - _cols.append(self[c]) + _cols.append(F.col(c)) elif isinstance(c, int) and not isinstance(c, bool): if c < 1: raise PySparkIndexError( @@ -675,7 +675,7 @@ def groupingSets( if isinstance(c, Column): gset.append(c) elif isinstance(c, str): - gset.append(self[c]) + gset.append(F.col(c)) else: raise PySparkTypeError( errorClass="NOT_COLUMN_OR_STR", @@ -691,7 +691,7 @@ def groupingSets( if isinstance(c, Column): gcols.append(c) elif isinstance(c, str): - gcols.append(self[c]) + gcols.append(F.col(c)) else: raise PySparkTypeError( errorClass="NOT_COLUMN_OR_STR", From 3c2379445e15e711a36f4a4ea1bc176f7febfc75 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Fri, 14 Nov 2025 08:01:30 -0400 Subject: [PATCH 136/400] [SPARK-54194][CONNECT][FOLLOWUP] Spark Connect Proto Plan Compression - Scala Client ### What changes were proposed in this pull request? In the previous PR https://github.com/apache/spark/pull/52894 of Spark Connect Proto Plan Compression, both Server-side and PySpark client changes were implemented. In this PR, the corresponding Scala client changes are implemented, so plan compression are now supported on the Scala client as well. To reproduce the existing issue we are solving here, run this code on Spark Connect Scala client: ``` import scala.util.Random import org.apache.spark.sql.DataFrame import spark.implicits._ def randomLetters(n: Int): String = { Iterator.continually(Random.nextPrintableChar()) .filter(_.isLetter) .take(n) .mkString } val numUniqueSmallRelations = 5 val sizePerSmallRelation = 512 * 1024 val smallDfs: Seq[DataFrame] = (0 until numUniqueSmallRelations).map { _ => Seq(randomLetters(sizePerSmallRelation)).toDF("value") } var resultDf = smallDfs.head for (_ <- 0 until 500) { val idx = Random.nextInt(smallDfs.length) resultDf = resultDf.unionByName(smallDfs(idx)) } resultDf.collect() ``` It fails with RESOURCE_EXHAUSTED error with message `gRPC message exceeds maximum size 134217728: 269207219`, because the server is trying to send an ExecutePlanResponse of ~260MB to the client. With the improvement introduced by the PR, the above code runs successfully and prints the expected result. ### Why are the changes needed? It improves Spark Connect stability when handling large plans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53003 from xi-db/plan-compression-scala-client. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell (cherry picked from commit 6cb88c10126bde79076ce5c8d7574cc5c9524746) Signed-off-by: Herman van Hovell --- .../docs/source/getting_started/install.rst | 2 + sql/connect/client/jvm/pom.xml | 5 + .../sql/connect/ClientE2ETestSuite.scala | 18 +- .../client/SparkConnectClientSuite.scala | 134 +++++++ sql/connect/common/pom.xml | 4 + .../connect/client/SparkConnectClient.scala | 362 +++++++++++++----- .../spark/sql/connect/config/Connect.scala | 3 +- .../service/SparkConnectServiceE2ESuite.scala | 2 +- 8 files changed, 436 insertions(+), 94 deletions(-) diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index 8b3c969d756d9..6b5a09205e4aa 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -230,6 +230,7 @@ Package Supported version Note `grpcio` >=1.76.0 Required for Spark Connect `grpcio-status` >=1.76.0 Required for Spark Connect `googleapis-common-protos` >=1.71.0 Required for Spark Connect +`zstandard` >=0.25.0 Required for Spark Connect `graphviz` >=0.20 Optional for Spark Connect ========================== ================= ========================== @@ -313,6 +314,7 @@ Package Supported version Note `grpcio` >=1.76.0 Required for Spark Connect `grpcio-status` >=1.76.0 Required for Spark Connect `googleapis-common-protos` >=1.71.0 Required for Spark Connect +`zstandard` >=0.25.0 Required for Spark Connect `pyyaml` >=3.11 Required for spark-pipelines command line interface `graphviz` >=0.20 Optional for Spark Connect ========================== ================= =================================================== diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index fb983040843ba..9514ed1ff202f 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -84,6 +84,11 @@ failureaccess compile + + com.github.luben + zstd-jni + compile + diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala index 773b4c375c0b0..338905a3302c6 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.StringEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connect.ConnectConversions._ -import org.apache.spark.sql.connect.client.{RetryPolicy, SparkConnectClient, SparkResult} +import org.apache.spark.sql.connect.client.{PlanCompressionOptions, RetryPolicy, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.test.{ConnectFunSuite, IntegrationTestUtils, QueryTest, RemoteSparkSession, SQLHelper} import org.apache.spark.sql.connect.test.SparkConnectServerUtils.{createSparkSession, port} import org.apache.spark.sql.functions._ @@ -2013,6 +2013,22 @@ class ClientE2ETestSuite } } } + + test("Plan compression works correctly") { + val originalPlanCompressionOptions = spark.client.getPlanCompressionOptions + assert(originalPlanCompressionOptions.nonEmpty) + assert(originalPlanCompressionOptions.get.thresholdBytes > 0) + assert(originalPlanCompressionOptions.get.algorithm == "ZSTD") + try { + spark.client.setPlanCompressionOptions(Some(PlanCompressionOptions(1000, "ZSTD"))) + // Execution should work + assert(spark.sql(s"select '${"Apache Spark" * 10000}' as value").collect().length == 1) + // Analysis should work + assert(spark.sql(s"select '${"Apache Spark" * 10000}' as value").columns.length == 1) + } finally { + spark.client.setPlanCompressionOptions(originalPlanCompressionOptions) + } + } } private[sql] case class ClassData(a: String, b: Int) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index a41ea344cbd4c..743112c6dd4dd 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -187,6 +187,9 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { .builder() .connectionString(s"sc://localhost:${server.getPort}") .build() + // Disable plan compression to make sure there is only one RPC request in client.analyze, + // so the interceptor can capture the initial header. + client.setPlanCompressionOptions(None) val session = SparkSession.builder().client(client).create() val df = session.range(10) @@ -521,6 +524,9 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { .connectionString(s"sc://localhost:${server.getPort}") .enableReattachableExecute() .build() + // Disable plan compression to make sure there is only one RPC request in client.analyze, + // so the interceptor can capture the initial header. + client.setPlanCompressionOptions(None) val plan = buildPlan("select * from range(10000000)") val dummyUUID = "10a4c38e-7e87-40ee-9d6f-60ff0751e63b" @@ -533,6 +539,87 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { assert(resp.getOperationId == dummyUUID) } } + + test("Plan compression works correctly for execution") { + startDummyServer(0) + client = SparkConnectClient + .builder() + .connectionString(s"sc://localhost:${server.getPort}") + .enableReattachableExecute() + .build() + // Set plan compression options for testing + client.setPlanCompressionOptions(Some(PlanCompressionOptions(1000, "ZSTD"))) + + // Small plan should not be compressed + val plan = buildPlan("select * from range(10)") + val iter = client.execute(plan) + val reattachableIter = + ExecutePlanResponseReattachableIterator.fromIterator(iter) + while (reattachableIter.hasNext) { + reattachableIter.next() + } + assert(service.getAndClearLatestInputPlan().hasRoot) + + // Large plan should be compressed + val plan2 = buildPlan(s"select ${"Apache Spark" * 10000} as value") + val iter2 = client.execute(plan2) + val reattachableIter2 = + ExecutePlanResponseReattachableIterator.fromIterator(iter2) + while (reattachableIter2.hasNext) { + reattachableIter2.next() + } + assert(service.getAndClearLatestInputPlan().hasCompressedOperation) + } + + test("Plan compression works correctly for analysis") { + startDummyServer(0) + client = SparkConnectClient + .builder() + .connectionString(s"sc://localhost:${server.getPort}") + .enableReattachableExecute() + .build() + // Set plan compression options for testing + client.setPlanCompressionOptions(Some(PlanCompressionOptions(1000, "ZSTD"))) + + // Small plan should not be compressed + val plan = buildPlan("select * from range(10)") + client.analyze(proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA, Some(plan)) + assert(service.getAndClearLatestInputPlan().hasRoot) + + // Large plan should be compressed + val plan2 = buildPlan(s"select ${"Apache Spark" * 10000} as value") + client.analyze(proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA, Some(plan2)) + assert(service.getAndClearLatestInputPlan().hasCompressedOperation) + } + + test("Plan compression will be disabled if the configs are not defined on the server") { + startDummyServer(0) + client = SparkConnectClient + .builder() + .connectionString(s"sc://localhost:${server.getPort}") + .enableReattachableExecute() + .build() + + service.setErrorToThrowOnConfig( + "spark.connect.session.planCompression.defaultAlgorithm", + new StatusRuntimeException(Status.INTERNAL.withDescription("SQL_CONF_NOT_FOUND"))) + + // Execute a few queries to make sure the client fetches the configs only once. + (1 to 3).foreach { _ => + val plan = buildPlan(s"select ${"Apache Spark" * 10000} as value") + val iter = client.execute(plan) + val reattachableIter = + ExecutePlanResponseReattachableIterator.fromIterator(iter) + while (reattachableIter.hasNext) { + reattachableIter.next() + } + assert(service.getAndClearLatestInputPlan().hasRoot) + } + // The plan compression options should be empty. + assert(client.getPlanCompressionOptions.isEmpty) + // The client should try to fetch the config only once. + assert(service.getAndClearLatestConfigRequests().size == 1) + } } class DummySparkConnectService() extends SparkConnectServiceGrpc.SparkConnectServiceImplBase { @@ -540,9 +627,17 @@ class DummySparkConnectService() extends SparkConnectServiceGrpc.SparkConnectSer private var inputPlan: proto.Plan = _ private val inputArtifactRequests: mutable.ListBuffer[AddArtifactsRequest] = mutable.ListBuffer.empty + private val inputConfigRequests = mutable.ListBuffer.empty[proto.ConfigRequest] + private val sparkConfigs = mutable.Map.empty[String, String] var errorToThrowOnExecute: Option[Throwable] = None + private var errorToThrowOnConfig: Map[String, Throwable] = Map.empty + + private[sql] def setErrorToThrowOnConfig(key: String, error: Throwable): Unit = synchronized { + errorToThrowOnConfig = errorToThrowOnConfig + (key -> error) + } + private[sql] def getAndClearLatestInputPlan(): proto.Plan = synchronized { val plan = inputPlan inputPlan = null @@ -556,6 +651,13 @@ class DummySparkConnectService() extends SparkConnectServiceGrpc.SparkConnectSer requests } + private[sql] def getAndClearLatestConfigRequests(): Seq[proto.ConfigRequest] = + synchronized { + val requests = inputConfigRequests.clone().toSeq + inputConfigRequests.clear() + requests + } + override def executePlan( request: ExecutePlanRequest, responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { @@ -666,6 +768,38 @@ class DummySparkConnectService() extends SparkConnectServiceGrpc.SparkConnectSer responseObserver.onCompleted() } + override def config( + request: proto.ConfigRequest, + responseObserver: StreamObserver[proto.ConfigResponse]): Unit = { + inputConfigRequests.synchronized { + inputConfigRequests.append(request) + } + require( + request.getOperation.hasGetOption, + "Only GetOption is supported. Other operations " + + "can be implemented by following the same procedure below.") + + val responseBuilder = proto.ConfigResponse.newBuilder().setSessionId(request.getSessionId) + request.getOperation.getGetOption.getKeysList.asScala.iterator.foreach { key => + if (errorToThrowOnConfig.contains(key)) { + val error = errorToThrowOnConfig(key) + responseObserver.onError(error) + return + } + + val kvBuilder = proto.KeyValue.newBuilder() + synchronized { + sparkConfigs.get(key).foreach { value => + kvBuilder.setKey(key) + kvBuilder.setValue(value) + } + } + responseBuilder.addPairs(kvBuilder.build()) + } + responseObserver.onNext(responseBuilder.build()) + responseObserver.onCompleted() + } + override def interrupt( request: proto.InterruptRequest, responseObserver: StreamObserver[proto.InterruptResponse]): Unit = { diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 7e91b056de551..4b7df7a232d34 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -87,6 +87,10 @@ netty-transport-native-unix-common ${netty.version} + + com.github.luben + zstd-jni + - 4.2.33 + 4.2.37 1.12.1 1.15.3 From 9b23cf84c065803503a148372b9aaf6b3e9d91eb Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 14 Nov 2025 07:56:14 -0800 Subject: [PATCH 138/400] [SPARK-54114][CONNECT] Support getColumns for SparkConnectDatabaseMetaData ### What changes were proposed in this pull request? This PR implements the [`java.sql.DatabaseMetaData#getColumns`]( https://docs.oracle.com/en/java/javase/17/docs/api/java.sql/java/sql/DatabaseMetaData.html#getColumns(java.lang.String,java.lang.String,java.lang.String,java.lang.String)) for `SparkConnectDatabaseMetaData` ### Why are the changes needed? Improve JDBC API coverage for Connect JDBC driver. ### Does this PR introduce _any_ user-facing change? No, the Connect JDBC driver is a new feature under development. ### How was this patch tested? New UTs. Also tested with BeeLine, and DBeaver ``` 0: jdbc:sc://localhost:15002> use tpch_sf1; No rows affected (0.042 seconds) 0: jdbc:sc://localhost:15002> !columns customer +----------------+--------------+-------------+-------------------------+------------+------------+--------------+----------------+-----------------+-----------------+-----------+----------+-------------+----------------+-------------------+--------------------+-------------------+--------------+----------------+---------------+--------------+-------------------+-------------------+---------------------+ | TABLE_CAT | TABLE_SCHEM | TABLE_NAME | COLUMN_NAME | DATA_TYPE | TYPE_NAME | COLUMN_SIZE | BUFFER_LENGTH | DECIMAL_DIGITS | NUM_PREC_RADIX | NULLABLE | REMARKS | COLUMN_DEF | SQL_DATA_TYPE | SQL_DATETIME_SUB | CHAR_OCTET_LENGTH | ORDINAL_POSITION | IS_NULLABLE | SCOPE_CATALOG | SCOPE_SCHEMA | SCOPE_TABLE | SOURCE_DATA_TYPE | IS_AUTOINCREMENT | IS_GENERATEDCOLUMN | +----------------+--------------+-------------+-------------------------+------------+------------+--------------+----------------+-----------------+-----------------+-----------+----------+-------------+----------------+-------------------+--------------------+-------------------+--------------+----------------+---------------+--------------+-------------------+-------------------+---------------------+ | spark_catalog | tpcds_sf1 | customer | c_customer_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 1 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_customer_id | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 2 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_current_cdemo_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 3 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_current_hdemo_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 4 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_current_addr_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 5 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_first_shipto_date_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 6 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_first_sales_date_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 7 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_salutation | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 8 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_first_name | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 9 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_last_name | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 10 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_preferred_cust_flag | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 11 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_birth_day | 4 | INT | 11 | 0 | 0 | 10 | 1 | | NULL | 0 | 0 | 0 | 12 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_birth_month | 4 | INT | 11 | 0 | 0 | 10 | 1 | | NULL | 0 | 0 | 0 | 13 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_birth_year | 4 | INT | 11 | 0 | 0 | 10 | 1 | | NULL | 0 | 0 | 0 | 14 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_birth_country | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 15 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_login | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 16 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_email_address | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 17 | YES | | | | 0 | | | | spark_catalog | tpcds_sf1 | customer | c_last_review_date_sk | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 18 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_custkey | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 1 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_name | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 2 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_address | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 3 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_nationkey | -5 | BIGINT | 20 | 0 | NULL | 10 | 1 | | NULL | 0 | 0 | 0 | 4 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_phone | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 5 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_acctbal | 8 | DOUBLE | 24 | 0 | 15 | 10 | 1 | | NULL | 0 | 0 | 0 | 6 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_mktsegment | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 7 | YES | | | | 0 | | | | spark_catalog | tpch_sf1 | customer | c_comment | 12 | STRING | 255 | 0 | NULL | NULL | 1 | | NULL | 0 | 0 | 0 | 8 | YES | | | | 0 | | | +----------------+--------------+-------------+-------------------------+------------+------------+--------------+----------------+-----------------+-----------------+-----------+----------+-------------+----------------+-------------------+--------------------+-------------------+--------------+----------------+---------------+--------------+-------------------+-------------------+---------------------+ ``` image ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53060 from pan3793/SPARK-54114. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit be98a29875cecb83d4a62d560d47b35cbd4c6271) Signed-off-by: Dongjoon Hyun --- .../jdbc/SparkConnectDatabaseMetaData.scala | 89 ++++++- .../client/jdbc/util/JdbcTypeUtils.scala | 15 ++ .../SparkConnectDatabaseMetaDataSuite.scala | 239 ++++++++++++++++++ 3 files changed, 341 insertions(+), 2 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala index 47dfef2eb7058..490fb7b6472ea 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connect.client.jdbc import java.sql.{Array => _, _} +import java.sql.DatabaseMetaData._ import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} import org.apache.spark.SparkThrowable @@ -25,6 +26,7 @@ import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.util.QuotingUtils._ import org.apache.spark.sql.connect import org.apache.spark.sql.connect.client.jdbc.SparkConnectDatabaseMetaData._ +import org.apache.spark.sql.connect.client.jdbc.util.JdbcTypeUtils import org.apache.spark.sql.functions._ import org.apache.spark.util.VersionUtils @@ -479,8 +481,91 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas catalog: String, schemaPattern: String, tableNamePattern: String, - columnNamePattern: String): ResultSet = - throw new SQLFeatureNotSupportedException + columnNamePattern: String): ResultSet = { + conn.checkOpen() + + val columnNameFilterExpr = if (isNullOrWildcard(columnNamePattern)) { + lit(true) + } else { + $"COLUMN_NAME".like(columnNamePattern) + } + + val emptyDf = conn.spark.emptyDataFrame + .withColumn("TABLE_CAT", lit("")) + .withColumn("TABLE_SCHEM", lit("")) + .withColumn("TABLE_NAME", lit("")) + .withColumn("COLUMN_NAME", lit("")) + .withColumn("DATA_TYPE", lit(0)) + .withColumn("TYPE_NAME", lit("")) + .withColumn("COLUMN_SIZE", lit(0)) + .withColumn("BUFFER_LENGTH", lit(0)) + .withColumn("DECIMAL_DIGITS", lit(0)) + .withColumn("NUM_PREC_RADIX", lit(0)) + .withColumn("NULLABLE", lit(0)) + .withColumn("REMARKS", lit("")) + .withColumn("COLUMN_DEF", lit("")) + .withColumn("SQL_DATA_TYPE", lit(0)) + .withColumn("SQL_DATETIME_SUB", lit(0)) + .withColumn("CHAR_OCTET_LENGTH", lit(0)) + .withColumn("ORDINAL_POSITION", lit(0)) + .withColumn("IS_NULLABLE", lit("")) + .withColumn("SCOPE_CATALOG", lit("")) + .withColumn("SCOPE_SCHEMA", lit("")) + .withColumn("SCOPE_TABLE", lit("")) + .withColumn("SOURCE_DATA_TYPE", lit(0.toShort)) + .withColumn("IS_AUTOINCREMENT", lit("")) + .withColumn("IS_GENERATEDCOLUMN", lit("")) + + val catalogSchemaTables = + getTablesDataFrame(catalog, schemaPattern, tableNamePattern) + .select("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME") + .collect().map { row => (row.getString(0), row.getString(1), row.getString(2)) } + + val df = catalogSchemaTables.map { case (catalog, schema, table) => + val columns = conn.spark.table(quoteNameParts(Seq(catalog, schema, table))) + .schema.zipWithIndex.map { case (field, i) => + ( + field.name, // COLUMN_NAME + JdbcTypeUtils.getColumnType(field), // DATA_TYPE + field.dataType.sql, // TYPE_NAME + JdbcTypeUtils.getDisplaySize(field), // COLUMN_SIZE + JdbcTypeUtils.getDecimalDigits(field), // DECIMAL_DIGITS + JdbcTypeUtils.getNumPrecRadix(field), // NUM_PREC_RADIX + if (field.nullable) columnNullable else columnNoNulls, // NULLABLE + field.getComment().orNull, // REMARKS + field.getCurrentDefaultValue().orNull, // COLUMN_DEF + 0, // CHAR_OCTET_LENGTH + i + 1, // ORDINAL_POSITION + if (field.nullable) "YES" else "NO", // IS_NULLABLE + "", // IS_AUTOINCREMENT + "" // IS_GENERATEDCOLUMN + ) + } + columns.toDF("COLUMN_NAME", "DATA_TYPE", "TYPE_NAME", "COLUMN_SIZE", "DECIMAL_DIGITS", + "NUM_PREC_RADIX", "NULLABLE", "REMARKS", "COLUMN_DEF", "CHAR_OCTET_LENGTH", + "ORDINAL_POSITION", "IS_NULLABLE", "IS_AUTOINCREMENT", "IS_GENERATEDCOLUMN") + .filter(columnNameFilterExpr) + .withColumn("TABLE_CAT", lit(catalog)) + .withColumn("TABLE_SCHEM", lit(schema)) + .withColumn("TABLE_NAME", lit(table)) + .withColumn("BUFFER_LENGTH", lit(0)) + .withColumn("SQL_DATA_TYPE", lit(0)) + .withColumn("SQL_DATETIME_SUB", lit(0)) + .withColumn("SCOPE_CATALOG", lit("")) + .withColumn("SCOPE_SCHEMA", lit("")) + .withColumn("SCOPE_TABLE", lit("")) + .withColumn("SOURCE_DATA_TYPE", lit(0.toShort)) + .select("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "DATA_TYPE", + "TYPE_NAME", "COLUMN_SIZE", "BUFFER_LENGTH", "DECIMAL_DIGITS", "NUM_PREC_RADIX", + "NULLABLE", "REMARKS", "COLUMN_DEF", "SQL_DATA_TYPE", "SQL_DATETIME_SUB", + "CHAR_OCTET_LENGTH", "ORDINAL_POSITION", "IS_NULLABLE", "SCOPE_CATALOG", + "SCOPE_SCHEMA", "SCOPE_TABLE", "SOURCE_DATA_TYPE", "IS_AUTOINCREMENT", + "IS_GENERATEDCOLUMN") + }.fold(emptyDf) { (l, r) => l.unionAll(r) } + .orderBy("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "ORDINAL_POSITION") + + new SparkConnectResultSet(df.collectResult()) + } override def getColumnPrivileges( catalog: String, diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala index 91de849e46c03..a3adf2b180d85 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcTypeUtils.scala @@ -134,4 +134,19 @@ private[jdbc] object JdbcTypeUtils { case other => throw new SQLFeatureNotSupportedException(s"DataType $other is not supported yet.") } + + def getDecimalDigits(field: StructField): Integer = field.dataType match { + case BooleanType | _: IntegralType => 0 + case FloatType => 7 + case DoubleType => 15 + case d: DecimalType => d.scale + case TimeType(scale) => scale + case TimestampType | TimestampNTZType => 6 + case _ => null + } + + def getNumPrecRadix(field: StructField): Integer = field.dataType match { + case _: NumericType => 10 + case _ => null + } } diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala index 3b6622f97a3e4..255537af4bbb4 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala @@ -547,4 +547,243 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark } } } + + test("SparkConnectDatabaseMetaData getColumns") { + + case class GetColumnResult( + TABLE_CAT: String, + TABLE_SCHEM: String, + TABLE_NAME: String, + COLUMN_NAME: String, + DATA_TYPE: Int, + TYPE_NAME: String, + COLUMN_SIZE: Int, + BUFFER_LENGTH: Int, + DECIMAL_DIGITS: Int, + NUM_PREC_RADIX: Int, + NULLABLE: Int, + REMARKS: String, + COLUMN_DEF: String, + SQL_DATA_TYPE: Int, + SQL_DATETIME_SUB: Int, + CHAR_OCTET_LENGTH: Int, + ORDINAL_POSITION: Int, + IS_NULLABLE: String, + SCOPE_CATALOG: String, + SCOPE_SCHEMA: String, + SCOPE_TABLE: String, + SOURCE_DATA_TYPE: Short, + IS_AUTOINCREMENT: String, + IS_GENERATEDCOLUMN: String) + + def verifyEmptyFields(result: GetColumnResult): Unit = { + assert(result.BUFFER_LENGTH === 0) + assert(result.SQL_DATA_TYPE === 0) + assert(result.SQL_DATETIME_SUB === 0) + assert(result.SCOPE_CATALOG === "") + assert(result.SCOPE_SCHEMA === "") + assert(result.SCOPE_TABLE === "") + assert(result.SOURCE_DATA_TYPE === 0.toShort) + } + + def verifyGetColumns( + getColumns: () => ResultSet)(verify: Seq[GetColumnResult] => Unit): Unit = { + Using.resource(getColumns()) { rs => + val getTableResults = new Iterator[GetColumnResult] { + def hasNext: Boolean = rs.next() + + def next(): GetColumnResult = GetColumnResult( + TABLE_CAT = rs.getString("TABLE_CAT"), + TABLE_SCHEM = rs.getString("TABLE_SCHEM"), + TABLE_NAME = rs.getString("TABLE_NAME"), + COLUMN_NAME = rs.getString("COLUMN_NAME"), + DATA_TYPE = rs.getInt("DATA_TYPE"), + TYPE_NAME = rs.getString("TYPE_NAME"), + COLUMN_SIZE = rs.getInt("COLUMN_SIZE"), + BUFFER_LENGTH = rs.getInt("BUFFER_LENGTH"), + DECIMAL_DIGITS = rs.getInt("DECIMAL_DIGITS"), + NUM_PREC_RADIX = rs.getInt("NUM_PREC_RADIX"), + NULLABLE = rs.getInt("NULLABLE"), + REMARKS = rs.getString("REMARKS"), + COLUMN_DEF = rs.getString("COLUMN_DEF"), + SQL_DATA_TYPE = rs.getInt("SQL_DATA_TYPE"), + SQL_DATETIME_SUB = rs.getInt("SQL_DATETIME_SUB"), + CHAR_OCTET_LENGTH = rs.getInt("CHAR_OCTET_LENGTH"), + ORDINAL_POSITION = rs.getInt("ORDINAL_POSITION"), + IS_NULLABLE = rs.getString("IS_NULLABLE"), + SCOPE_CATALOG = rs.getString("SCOPE_CATALOG"), + SCOPE_SCHEMA = rs.getString("SCOPE_SCHEMA"), + SCOPE_TABLE = rs.getString("SCOPE_TABLE"), + SOURCE_DATA_TYPE = rs.getShort("SOURCE_DATA_TYPE"), + IS_AUTOINCREMENT = rs.getString("IS_AUTOINCREMENT"), + IS_GENERATEDCOLUMN = rs.getString("IS_GENERATEDCOLUMN")) + }.toSeq + verify(getTableResults) + } + } + + withConnection { conn => + implicit val spark: SparkSession = conn.asInstanceOf[SparkConnectConnection].spark + + spark.sql("CREATE DATABASE IF NOT EXISTS testcat.t_db1") + spark.sql("CREATE TABLE IF NOT EXISTS testcat.t_db1.t_t1 (id INT)") + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db1") + spark.sql( + """CREATE TABLE IF NOT EXISTS spark_catalog.db1.t1 ( + | id INT NOT NULL, + | i_ INT, + | location STRING COMMENT 'city name' DEFAULT 'unknown') + |""".stripMargin) + spark.sql( + """CREATE TABLE IF NOT EXISTS spark_catalog.db1.t2 ( + | col_null VOID, + | col_boolean BOOLEAN, + | col_byte BYTE, + | col_short SHORT, + | col_int INT, + | col_long LONG, + | col_float FLOAT, + | col_double DOUBLE, + | col_string STRING, + | col_decimal DECIMAL(10, 5), + | col_date DATE, + | col_timestamp TIMESTAMP, + | col_timestamp_ntz TIMESTAMP_NTZ, + | col_binary BINARY, + | col_time TIME)""".stripMargin) + + spark.sql( + """CREATE VIEW IF NOT EXISTS spark_catalog.db1.t1_v AS + |SELECT id FROM spark_catalog.db1.t1 + |""".stripMargin) + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db_") + spark.sql("CREATE TABLE IF NOT EXISTS spark_catalog.db_.t_ (id INT, i_ INT)") + + spark.sql("CREATE DATABASE IF NOT EXISTS spark_catalog.db_2") + spark.sql("CREATE TABLE IF NOT EXISTS spark_catalog.db_2.t_2 (id INT)") + + val metadata = conn.getMetaData + + // no need to care about "testcat" because it is memory based and session isolated, + // also is inaccessible from another SparkSession + withDatabase("spark_catalog.db1", "spark_catalog.db_2", "spark_catalog.db_") { + // list columns of all tables in all catalogs and schemas + val getColumnsInAllTables = List(null, "%").flatMap { database => + List(null, "%").flatMap { table => + List(null, "%").map { column => + () => metadata.getColumns(null, database, table, column) + } + } + } + + getColumnsInAllTables.foreach { getColumns => + verifyGetColumns(getColumns) { getColumnResults => + // results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, ORDINAL_POSITION + assert { + getColumnResults.map { r => + (r.TABLE_CAT, r.TABLE_SCHEM, r.TABLE_NAME, r.ORDINAL_POSITION, r.COLUMN_NAME) + } === Seq( + ("spark_catalog", "db1", "t1", 1, "id"), + ("spark_catalog", "db1", "t1", 2, "i_"), + ("spark_catalog", "db1", "t1", 3, "location"), + ("spark_catalog", "db1", "t1_v", 1, "id"), + ("spark_catalog", "db1", "t2", 1, "col_null"), + ("spark_catalog", "db1", "t2", 2, "col_boolean"), + ("spark_catalog", "db1", "t2", 3, "col_byte"), + ("spark_catalog", "db1", "t2", 4, "col_short"), + ("spark_catalog", "db1", "t2", 5, "col_int"), + ("spark_catalog", "db1", "t2", 6, "col_long"), + ("spark_catalog", "db1", "t2", 7, "col_float"), + ("spark_catalog", "db1", "t2", 8, "col_double"), + ("spark_catalog", "db1", "t2", 9, "col_string"), + ("spark_catalog", "db1", "t2", 10, "col_decimal"), + ("spark_catalog", "db1", "t2", 11, "col_date"), + ("spark_catalog", "db1", "t2", 12, "col_timestamp"), + ("spark_catalog", "db1", "t2", 13, "col_timestamp_ntz"), + ("spark_catalog", "db1", "t2", 14, "col_binary"), + ("spark_catalog", "db1", "t2", 15, "col_time"), + ("spark_catalog", "db_", "t_", 1, "id"), + ("spark_catalog", "db_", "t_", 2, "i_"), + ("spark_catalog", "db_2", "t_2", 1, "id"), + ("testcat", "t_db1", "t_t1", 1, "id")) + } + + // TODO verify the remaining attributes + // DATA_TYPE = rs.getInt("DATA_TYPE"), + // TYPE_NAME = rs.getString("TYPE_NAME"), + // COLUMN_SIZE = rs.getInt("COLUMN_SIZE"), + // DECIMAL_DIGITS = rs.getInt("DECIMAL_DIGITS"), + // NUM_PREC_RADIX = rs.getInt("NUM_PREC_RADIX"), + // NULLABLE = rs.getInt("NULLABLE"), + // REMARKS = rs.getString("REMARKS"), + // COLUMN_DEF = rs.getString("COLUMN_DEF"), + // CHAR_OCTET_LENGTH = rs.getInt("CHAR_OCTET_LENGTH"), + // IS_NULLABLE = rs.getString("IS_NULLABLE"), + // IS_AUTOINCREMENT = rs.getString("IS_AUTOINCREMENT"), + // IS_GENERATEDCOLUMN = rs.getString("IS_GENERATEDCOLUMN") + + getColumnResults.foreach(verifyEmptyFields) + } + } + + // list columns of all tables in the current catalog and schema + conn.setCatalog("spark_catalog") + conn.setSchema("db1") + assert(conn.getCatalog === "spark_catalog") + assert(conn.getSchema === "db1") + + verifyGetColumns(() => metadata.getColumns("", "", "%", "id")) { getColumnResults => + // results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, ORDINAL_POSITION + assert { + getColumnResults.map { r => + (r.TABLE_CAT, r.TABLE_SCHEM, r.TABLE_NAME, r.ORDINAL_POSITION, r.COLUMN_NAME) + } === Seq( + ("spark_catalog", "db1", "t1", 1, "id"), + ("spark_catalog", "db1", "t1_v", 1, "id")) + } + + getColumnResults.foreach(verifyEmptyFields) + } + + // list columns of tables with schema pattern, table mame pattern, and column name pattern + verifyGetColumns { + () => metadata.getColumns(null, "%db_", "%t_", "%d%") + } { getColumnResults => + // results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, ORDINAL_POSITION + assert { + getColumnResults.map { r => + (r.TABLE_CAT, r.TABLE_SCHEM, r.TABLE_NAME, r.ORDINAL_POSITION, r.COLUMN_NAME) + } === Seq( + ("spark_catalog", "db1", "t1", 1, "id"), + ("spark_catalog", "db1", "t2", 8, "col_double"), + ("spark_catalog", "db1", "t2", 10, "col_decimal"), + ("spark_catalog", "db1", "t2", 11, "col_date"), + ("spark_catalog", "db_", "t_", 1, "id"), + ("testcat", "t_db1", "t_t1", 1, "id")) + } + + getColumnResults.foreach(verifyEmptyFields) + } + + // escape _ in schema pattern and table mame pattern + verifyGetColumns { + () => metadata.getColumns(null, "db\\_", "t\\_", "i\\_") + } { getColumnResults => + // results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, ORDINAL_POSITION + assert { + getColumnResults.map { r => + (r.TABLE_CAT, r.TABLE_SCHEM, r.TABLE_NAME, r.ORDINAL_POSITION, r.COLUMN_NAME) + } === Seq(("spark_catalog", "db_", "t_", 2, "i_")) + } + + getColumnResults.foreach(verifyEmptyFields) + } + + // skip testing escape ', % in schema pattern, because Spark SQL does not + // allow using those chars in schema table name. + } + } + } } From 2d6c256f9325fa60347e2759a8a6b89481e35a2e Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 14 Nov 2025 09:36:47 -0800 Subject: [PATCH 139/400] [SPARK-54157][SQL] Fix refresh of DSv2 tables in Dataset ### What changes were proposed in this pull request? This PR fixes refresh of DSv2 tables in Dataset. ### Why are the changes needed? Prior to this change, Spark would pin the version of DSv2 tables at load/resolution time. Any changes within the session will not be propagated to the analyzed but not yet executed Dataset, breaking the behavior compared to DSv1 tables. Changes in this PR are needed for the following reasons: - Prevent scanning/joining inconsistent versions of the table in the same session (see tests). - Prevent stale results upon external and session changes. - Remove workarounds in DSv2 connectors by fixing the problem in Spark. ### Does this PR introduce _any_ user-facing change? Yes, but this PR makes DSv2 Table behavior match the expected Spark semantics. ### How was this patch tested? This PR comes with tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52920 from aokolnychyi/spark-54157. Authored-by: Anton Okolnychyi Signed-off-by: Wenchen Fan (cherry picked from commit 59759f3646cfecb1615d45742a15d98e24820af3) Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 25 + .../spark/sql/connector/catalog/Table.java | 10 + .../catalyst/plans/logical/v2Commands.scala | 10 +- .../sql/connector/catalog/V2TableUtil.scala | 180 ++++++++ .../sql/errors/QueryCompilationErrors.scala | 32 ++ .../datasources/v2/DataSourceV2Relation.scala | 12 +- .../datasources/v2/V2TableRefreshUtil.scala | 101 ++++ .../apache/spark/sql/util/SchemaUtils.scala | 7 + .../analysis/TableLookupCacheSuite.scala | 7 + .../sql/connector/catalog/InMemoryTable.scala | 17 +- .../catalog/InMemoryTableCatalog.scala | 20 +- .../connector/catalog/V2TableUtilSuite.scala | 431 ++++++++++++++++++ .../spark/sql/execution/QueryExecution.scala | 11 +- .../DataSourceV2DataFrameSuite.scala | 301 ++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 14 +- .../connector/MergeIntoDataFrameSuite.scala | 52 +++ .../sql/execution/SQLViewTestSuite.scala | 21 +- 17 files changed, 1234 insertions(+), 17 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 9fe18fac66fef..ca8e953a22460 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2205,6 +2205,31 @@ ], "sqlState" : "42000" }, + "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS" : { + "message" : [ + "Detected incompatible changes to table after DataFrame/Dataset has been resolved and analyzed, meaning the underlying plan is out of sync. Please, re-create DataFrame/Dataset before attempting to execute the query again." + ], + "subClass" : { + "COLUMNS_MISMATCH" : { + "message" : [ + "Data columns have changed:", + "" + ] + }, + "METADATA_COLUMNS_MISMATCH" : { + "message" : [ + "Metadata columns have changed:", + "" + ] + }, + "TABLE_ID_MISMATCH" : { + "message" : [ + "Table ID has changed from to ." + ] + } + }, + "sqlState" : "51024" + }, "INCOMPATIBLE_VIEW_SCHEMA_CHANGE" : { "message" : [ "The SQL query of view has an incompatible schema change and column cannot be resolved. Expected columns named but got .", diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index 3a1e0d9f7011e..f85b5e3569298 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -50,6 +50,16 @@ public interface Table { */ String name(); + /** + * An ID of the table that can be used to reliably check if two table objects refer to the same + * metastore entity. If a table is dropped and recreated again with the same name, the new table + * ID must be different. This method must return null if connectors don't support the notion of + * table ID. + */ + default String id() { + return null; + } + /** * Returns the schema of this table. If the table is not readable and doesn't have a schema, an * empty schema can be returned here. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index db5dacdcef38a..ea9f7c91cf005 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.connector.write.RowLevelOperation.Command.{DELETE, M import org.apache.spark.sql.errors.DataTypeErrors.toSQLType import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, ExtractV2Table} +import org.apache.spark.sql.execution.datasources.v2.V2TableRefreshUtil import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, AtomicType, BooleanType, DataType, IntegerType, MapType, MetadataBuilder, StringType, StructField, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -687,7 +688,14 @@ case class ReplaceTableAsSelect( isAnalyzed: Boolean = false) extends V2CreateTableAsSelectPlan { - override def markAsAnalyzed(ac: AnalysisContext): LogicalPlan = copy(isAnalyzed = true) + override def markAsAnalyzed(ac: AnalysisContext): LogicalPlan = { + // RTAS may drop and recreate table before query execution, breaking self-references + // refresh and pin versions here to read from original table versions instead of + // newly created empty table that is meant to serve as target for append/overwrite + val refreshedQuery = V2TableRefreshUtil.refreshVersions(query) + val pinnedQuery = V2TableRefreshUtil.pinVersions(refreshedQuery) + copy(query = pinnedQuery, isAnalyzed = true) + } override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { this.copy(partitioning = rewritten) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala new file mode 100644 index 0000000000000..9873fc0881a05 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.catalog + +import java.util.Locale + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, MetadataColumnHelper} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.util.SchemaUtils +import org.apache.spark.util.ArrayImplicits._ + +private[sql] object V2TableUtil extends SQLConfHelper { + + def toQualifiedName(catalog: CatalogPlugin, ident: Identifier): String = { + s"${quoteIfNeeded(catalog.name)}.${ident.quoted}" + } + + /** + * Validates that captured data columns match the current table schema. + * + * @param table the current table metadata + * @param relation the relation with captured columns + * @return validation errors, or empty sequence if valid + */ + def validateCapturedColumns(table: Table, relation: DataSourceV2Relation): Seq[String] = { + validateCapturedColumns(table, relation.table.columns.toImmutableArraySeq) + } + + /** + * Validates that captured data columns match the current table schema. + * + * Checks for: + * - Column type or nullability changes + * - Removed columns (missing from the current table schema) + * - Added columns (new in the current table schema) + * + * @param table the current table metadata + * @param originCols the originally captured columns + * @return validation errors, or empty sequence if valid + */ + def validateCapturedColumns(table: Table, originCols: Seq[Column]): Seq[String] = { + val errors = mutable.ArrayBuffer[String]() + val colsByNormalizedName = indexColumns(table.columns.toImmutableArraySeq) + val originColsByNormalizedName = indexColumns(originCols) + + originColsByNormalizedName.foreach { case (normalizedName, originCol) => + colsByNormalizedName.get(normalizedName) match { + case Some(col) => + if (originCol.dataType != col.dataType || originCol.nullable != col.nullable) { + val oldType = formatType(originCol.dataType, originCol.nullable) + val newType = formatType(col.dataType, col.nullable) + errors += s"`${originCol.name}` type has changed from $oldType to $newType" + } + case None => + errors += s"${formatColumn(originCol)} has been removed" + } + } + + colsByNormalizedName.foreach { case (normalizedName, col) => + if (!originColsByNormalizedName.contains(normalizedName)) { + errors += s"${formatColumn(col)} has been added" + } + } + + errors.toSeq + } + + /** + * Validates that captured metadata columns are consistent with the current table metadata. + * + * @param table the current table metadata + * @param relation the relation with captured metadata columns + * @return validation errors, or empty sequence if valid + */ + def validateCapturedMetadataColumns(table: Table, relation: DataSourceV2Relation): Seq[String] = { + validateCapturedMetadataColumns(table, extractMetadataColumns(relation)) + } + + // extracts original column info for all metadata attributes in relation + private def extractMetadataColumns(relation: DataSourceV2Relation): Seq[MetadataColumn] = { + val metaAttrs = relation.output.filter(_.isMetadataCol) + if (metaAttrs.nonEmpty) { + val metaCols = metadataColumns(relation.table) + val normalizedMetaAttrNames = metaAttrs.map(attr => normalize(attr.name)).toSet + metaCols.filter(col => normalizedMetaAttrNames.contains(normalize(col.name))) + } else { + Seq.empty + } + } + + /** + * Validates that captured metadata columns are consistent with the current table metadata. + * + * Checks for: + * - Metadata column type or nullability changes + * - Removed metadata columns (missing from current table) + * + * @param table the current table metadata + * @param originMetaCols the originally captured metadata columns + * @return validation errors, or empty sequence if valid + */ + def validateCapturedMetadataColumns( + table: Table, + originMetaCols: Seq[MetadataColumn]): Seq[String] = { + val errors = mutable.ArrayBuffer[String]() + val metaCols = metadataColumns(table) + val metaColsByNormalizedName = indexMetadataColumns(metaCols) + + originMetaCols.foreach { originMetaCol => + val normalizedName = normalize(originMetaCol.name) + metaColsByNormalizedName.get(normalizedName) match { + case Some(metaCol) => + if (originMetaCol.dataType != metaCol.dataType || + originMetaCol.isNullable != metaCol.isNullable) { + val oldType = formatType(originMetaCol.dataType, originMetaCol.isNullable) + val newType = formatType(metaCol.dataType, metaCol.isNullable) + errors += s"`${originMetaCol.name}` type has changed from $oldType to $newType" + } + case None => + errors += s"${formatMetadataColumn(originMetaCol)} has been removed" + } + } + + errors.toSeq + } + + private def formatColumn(col: Column): String = { + s"`${col.name}` ${formatType(col.dataType, col.nullable)}" + } + + private def formatMetadataColumn(col: MetadataColumn): String = { + s"`${col.name}` ${formatType(col.dataType, col.isNullable)}" + } + + private def formatType(dataType: DataType, nullable: Boolean): String = { + if (nullable) dataType.sql else s"${dataType.sql} NOT NULL" + } + + private def indexColumns(cols: Seq[Column]): Map[String, Column] = { + index(cols)(_.name) + } + + private def indexMetadataColumns(cols: Seq[MetadataColumn]): Map[String, MetadataColumn] = { + index(cols)(_.name) + } + + private def index[C](cols: Seq[C])(extractName: C => String): Map[String, C] = { + SchemaUtils.checkColumnNameDuplication(cols.map(extractName), conf.caseSensitiveAnalysis) + cols.map(col => normalize(extractName(col)) -> col).toMap + } + + private def metadataColumns(table: Table): Seq[MetadataColumn] = table match { + case hasMeta: SupportsMetadataColumns => hasMeta.metadataColumns.toImmutableArraySeq + case _ => Seq.empty + } + + private def normalize(name: String): String = { + if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 9381f2f6f2e34..29b87e2d00968 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2121,6 +2121,38 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat } } + def tableIdChangedAfterAnalysis( + tableName: String, + capturedTableId: String, + currentTableId: String): Throwable = { + new AnalysisException( + errorClass = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.TABLE_ID_MISMATCH", + messageParameters = Map( + "tableName" -> toSQLId(tableName), + "capturedTableId" -> capturedTableId, + "currentTableId" -> currentTableId)) + } + + def columnsChangedAfterAnalysis( + tableName: String, + errors: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", + messageParameters = Map( + "tableName" -> toSQLId(tableName), + "errors" -> errors.mkString("\n- ", "\n- ", ""))) + } + + def metadataColumnsChangedAfterAnalysis( + tableName: String, + errors: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.METADATA_COLUMNS_MISMATCH", + messageParameters = Map( + "tableName" -> toSQLId(tableName), + "errors" -> errors.mkString("\n- ", "\n- ", ""))) + } + def numberOfPartitionsNotAllowedWithUnspecifiedDistributionError(): Throwable = { new AnalysisException( errorClass = "INVALID_WRITE_DISTRIBUTION.PARTITION_NUM_WITH_UNSPECIFIED_DISTRIBUTION", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 9b8d48c3f3a85..3462ae0e4206e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -23,8 +23,9 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, ExposesMetadataColumns, Histogram, HistogramBin, LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, truncatedString, CharVarcharUtils} -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, FunctionCatalog, Identifier, SupportsMetadataColumns, Table, TableCapability} +import org.apache.spark.sql.catalyst.util.{truncatedString, CharVarcharUtils} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, FunctionCatalog, Identifier, SupportsMetadataColumns, Table, TableCapability, TableCatalog, V2TableUtil} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper import org.apache.spark.sql.connector.read.{Scan, Statistics => V2Statistics, SupportsReportStatistics} import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -57,9 +58,8 @@ abstract class DataSourceV2RelationBase( } override def name: String = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ (catalog, identifier) match { - case (Some(cat), Some(ident)) => s"${quoteIfNeeded(cat.name())}.${ident.quoted}" + case (Some(cat), Some(ident)) => V2TableUtil.toQualifiedName(cat, ident) case _ => table.name() } } @@ -259,10 +259,10 @@ object ExtractV2Table { } object ExtractV2CatalogAndIdentifier { - def unapply(relation: DataSourceV2Relation): Option[(CatalogPlugin, Identifier)] = { + def unapply(relation: DataSourceV2Relation): Option[(TableCatalog, Identifier)] = { relation match { case DataSourceV2Relation(_, _, Some(catalog), Some(identifier), _, _) => - Some((catalog, identifier)) + Some((catalog.asTableCatalog, identifier)) case _ => None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala new file mode 100644 index 0000000000000..4a21416292406 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import scala.collection.mutable + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.AsOfVersion +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog, V2TableUtil} +import org.apache.spark.sql.errors.QueryCompilationErrors + +private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { + /** + * Pins table versions for all versioned tables in the plan. + * + * This method captures the current version of each versioned table by adding time travel + * specifications. Tables that already have time travel specifications or are not versioned + * are left unchanged. + * + * @param plan the logical plan to pin versions for + * @return plan with pinned table versions + */ + def pinVersions(plan: LogicalPlan): LogicalPlan = { + plan transform { + case r @ ExtractV2CatalogAndIdentifier(catalog, ident) + if r.table.currentVersion != null && r.timeTravelSpec.isEmpty => + val tableName = V2TableUtil.toQualifiedName(catalog, ident) + val version = r.table.currentVersion + logDebug(s"Pinning table version for $tableName to $version") + r.copy(timeTravelSpec = Some(AsOfVersion(version))) + } + } + + /** + * Refreshes table metadata for all versioned tables in the plan. + * + * This method reloads table metadata from the catalog and validates: + * - Table identity: Ensures table ID has not changed + * - Data columns: Verifies captured columns match the current schema + * - Metadata columns: Checks metadata column consistency + * + * @param plan the logical plan to refresh + * @return plan with refreshed table metadata + */ + def refreshVersions(plan: LogicalPlan): LogicalPlan = { + val cache = mutable.HashMap.empty[(TableCatalog, Identifier), Table] + plan transform { + case r @ ExtractV2CatalogAndIdentifier(catalog, ident) + if r.table.currentVersion != null && r.timeTravelSpec.isEmpty => + val currentTable = cache.getOrElseUpdate((catalog, ident), { + val tableName = V2TableUtil.toQualifiedName(catalog, ident) + logDebug(s"Refreshing table metadata for $tableName") + catalog.loadTable(ident) + }) + validateTableIdentity(currentTable, r) + validateDataColumns(currentTable, r) + validateMetadataColumns(currentTable, r) + r.copy(table = currentTable) + } + } + + private def validateTableIdentity(currentTable: Table, relation: DataSourceV2Relation): Unit = { + if (relation.table.id != null && relation.table.id != currentTable.id) { + throw QueryCompilationErrors.tableIdChangedAfterAnalysis( + relation.name, + capturedTableId = relation.table.id, + currentTableId = currentTable.id) + } + } + + private def validateDataColumns(currentTable: Table, relation: DataSourceV2Relation): Unit = { + val errors = V2TableUtil.validateCapturedColumns(currentTable, relation) + if (errors.nonEmpty) { + throw QueryCompilationErrors.columnsChangedAfterAnalysis(relation.name, errors) + } + } + + private def validateMetadataColumns(currentTable: Table, relation: DataSourceV2Relation): Unit = { + val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation) + if (errors.nonEmpty) { + throw QueryCompilationErrors.metadataColumnsChangedAfterAnalysis(relation.name, errors) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index cfc17e2683ac8..42c7541e4c214 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -24,6 +24,8 @@ import scala.collection.immutable.Queue import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} +import org.apache.spark.sql.connector.catalog.CatalogV2Util +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types._ @@ -98,6 +100,11 @@ private[spark] object SchemaUtils { } } + def checkSchemaColumnNameDuplication(table: Table, resolver: Resolver): Unit = { + val schema = CatalogV2Util.v2ColumnsToStructType(table.columns) + checkSchemaColumnNameDuplication(schema, resolver) + } + /** * Checks if an input schema has duplicate column names. This throws an exception if the * duplication exists. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala index 189509e317364..9685ed5c6d256 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import java.io.File +import java.util import scala.jdk.CollectionConverters._ @@ -30,6 +31,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, InMemoryTable, InMemoryTableCatalog, Table} +import org.apache.spark.sql.connector.catalog.TableWritePrivilege import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.types._ @@ -55,6 +57,11 @@ class TableLookupCacheSuite extends AnalysisTest with Matchers { Array.empty, Map.empty[String, String].asJava) } + override def loadTable( + ident: Identifier, + writePrivileges: util.Set[TableWritePrivilege]): Table = { + loadTable(ident) + } override def name: String = CatalogManager.SESSION_CATALOG_NAME } val catalogManager = mock(classOf[CatalogManager]) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala index 46169a9db4914..beac7751bf775 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connector.catalog import java.util +import java.util.{Objects, UUID} import org.apache.spark.sql.connector.catalog.constraints.Constraint import org.apache.spark.sql.connector.distributions.{Distribution, Distributions} @@ -42,7 +43,8 @@ class InMemoryTable( numPartitions: Option[Int] = None, advisoryPartitionSize: Option[Long] = None, isDistributionStrictlyRequired: Boolean = true, - override val numRowsPerSplit: Int = Int.MaxValue) + override val numRowsPerSplit: Int = Int.MaxValue, + override val id: String = UUID.randomUUID().toString) extends InMemoryBaseTable(name, columns, partitioning, properties, constraints, distribution, ordering, numPartitions, advisoryPartitionSize, isDistributionStrictlyRequired, numRowsPerSplit) with SupportsDelete { @@ -137,7 +139,8 @@ class InMemoryTable( numPartitions, advisoryPartitionSize, isDistributionStrictlyRequired, - numRowsPerSplit) + numRowsPerSplit, + id) dataMap.synchronized { dataMap.foreach { case (key, splits) => @@ -160,6 +163,16 @@ class InMemoryTable( copiedTable } + override def equals(other: Any): Boolean = other match { + case that: InMemoryTable => + this.id == that.id && this.currentVersion() == that.currentVersion() + case _ => false + } + + override def hashCode(): Int = { + Objects.hash(id, currentVersion()) + } + class InMemoryWriterBuilderWithOverWrite(override val info: LogicalWriteInfo) extends InMemoryWriterBuilder(info) with SupportsOverwrite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index 1da0882ec211a..2156dba619eca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -46,9 +46,11 @@ class BasicInMemoryTableCatalog extends TableCatalog { private val invalidatedTables: util.Set[Identifier] = ConcurrentHashMap.newKeySet() private var _name: Option[String] = None + private var copyOnLoad: Boolean = false override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { _name = Some(name) + copyOnLoad = options.getBoolean("copyOnLoad", false) } override def name: String = _name.get @@ -57,7 +59,22 @@ class BasicInMemoryTableCatalog extends TableCatalog { tables.keySet.asScala.filter(_.namespace.sameElements(namespace)).toArray } + // load table for scans override def loadTable(ident: Identifier): Table = { + Option(tables.get(ident)) match { + case Some(table: InMemoryTable) if copyOnLoad => + table.copy() // copy to validate logical table equality + case Some(table) => + table + case _ => + throw new NoSuchTableException(ident.asMultipartIdentifier) + } + } + + // load table for writes + override def loadTable( + ident: Identifier, + writePrivileges: util.Set[TableWritePrivilege]): Table = { Option(tables.get(ident)) match { case Some(table) => table @@ -169,7 +186,8 @@ class BasicInMemoryTableCatalog extends TableCatalog { columns = CatalogV2Util.structTypeToV2Columns(schema), partitioning = finalPartitioning, properties = properties, - constraints = constraints) + constraints = constraints, + id = table.id) .alterTableWithData(table.data, schema) newTable.setCurrentVersion(currentVersion) changes.foreach { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala new file mode 100644 index 0000000000000..d484c8336603c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala @@ -0,0 +1,431 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.catalog + +import java.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, MetadataAttribute} +import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits.SparkArrayOps + +class V2TableUtilSuite extends SparkFunSuite { + + test("validateCapturedColumns - no changes") { + val cols = Array( + col("id", LongType, nullable = false), + col("name", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", cols) + + val errors = validateCapturedColumns(table, cols) + assert(errors.isEmpty, "No changes should produce no errors") + } + + test("validateCapturedColumns - column type changed") { + val originCols = Array( + col("id", LongType, nullable = true), // original type + col("name", StringType, nullable = true)) + val currentCols = Array( + col("id", StringType, nullable = true), // changed from LongType + col("name", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head.contains("`id` type has changed from BIGINT to STRING")) + } + + test("validateCapturedColumns - column nullability changed to not null") { + val originCols = Array( + col("id", LongType, nullable = true), // originally nullable + col("name", StringType, nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = false), // now NOT NULL + col("name", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`id` type has changed from BIGINT to BIGINT NOT NULL") + } + + test("validateCapturedColumns - column nullability changed to nullable") { + val originCols = Array( + col("id", LongType, nullable = false), // originally NOT NULL + col("name", StringType, nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), // now nullable + col("name", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`id` type has changed from BIGINT NOT NULL to BIGINT") + } + + test("validateCapturedColumns - column removed") { + val originCols = Array( + col("id", LongType, nullable = true), // originally present + col("name", StringType, nullable = true)) + val currentCols = Array( + col("name", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`id` BIGINT has been removed") + } + + test("validateCapturedColumns - column added") { + val originCols = Array( + col("id", LongType, nullable = true), + col("name", StringType, nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("name", StringType, nullable = true), + col("age", IntegerType, nullable = true)) // new column + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`age` INT has been added") + } + + test("validateCapturedColumns - multiple columns removed and added") { + val originCols = Array( + col("id", LongType, nullable = true), + col("name", StringType, nullable = true), // originally present + col("address", StringType, nullable = true)) // originally present + val currentCols = Array( + col("id", LongType, nullable = true), + col("email", StringType, nullable = true), // new column + col("age", IntegerType, nullable = true)) // new column + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 4) // 2 removed + 2 added + assert(errors.count(_.contains("removed")) == 2) + assert(errors.count(_.contains("has been added")) == 2) + } + + test("validateCapturedColumns - case insensitive column names") { + val originCols = Array( + col("id", LongType, nullable = true), // lowercase + col("name", StringType, nullable = true)) + val currentCols = Array( + col("ID", LongType, nullable = true), // uppercase + col("NAME", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.isEmpty, "Case insensitive comparison should match") + } + + test("validateCapturedColumns - duplicate columns with different case") { + val originCols = Array( + col("id", LongType, nullable = true), + col("name", StringType, nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("ID", StringType, nullable = true), // duplicate with different case + col("name", StringType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val e = intercept[AnalysisException] { validateCapturedColumns(table, originCols) } + assert(e.message.contains("Choose another name or rename the existing column")) + } + + test("validateCapturedColumns - complex types") { + val structType = StructType(Seq( + StructField("street", StringType), + StructField("city", StringType))) + val originCols = Array( + col("id", LongType, nullable = true), + col("address", structType, nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("address", structType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.isEmpty) + } + + test("validateCapturedColumns - complex type changed") { + val originStructType = StructType(Seq( + StructField("street", StringType), + StructField("city", StringType))) // originally StringType + val originCols = Array( + col("id", LongType, nullable = true), + col("address", originStructType, nullable = true)) + val currentStructType = StructType(Seq( + StructField("street", StringType), + StructField("city", IntegerType))) // changed type + val currentCols = Array( + col("id", LongType, nullable = true), + col("address", currentStructType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = V2TableUtil.validateCapturedColumns(table, originCols.toSeq) + assert(errors.size == 1) + assert(errors.head.contains( + "`address` type has changed from STRUCT to " + + "STRUCT")) + } + + test("validateCapturedMetadataColumns - no changes") { + val originMetaCols = Seq( + metaCol("_partition", StringType, nullable = false), + metaCol("index", IntegerType, nullable = false)) + val currentMetaCols = Array( + metaCol("_partition", StringType, nullable = false), + metaCol("index", IntegerType, nullable = false)) + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.isEmpty, "No changes should produce no errors") + } + + test("validateCapturedMetadataColumns - type changed") { + val originMetaCols = Seq( + metaCol("index", IntegerType, nullable = false)) // originally IntegerType + val currentMetaCols = Array( + metaCol("index", StringType, nullable = false)) // changed to StringType + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 1) + assert(errors.head == "`index` type has changed from INT NOT NULL to STRING NOT NULL") + } + + test("validateCapturedMetadataColumns - nullability changed to nullable") { + val originMetaCols = Seq( + metaCol("index", IntegerType, nullable = false)) // originally NOT NULL + val currentMetaCols = Array( + metaCol("index", IntegerType, nullable = true)) // now nullable + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 1) + assert(errors.head == "`index` type has changed from INT NOT NULL to INT") + } + + test("validateCapturedMetadataColumns - nullability changed to not null") { + val originMetaCols = Seq(metaCol("index", IntegerType, nullable = true)) // originally nullable + val currentMetaCols = Array(metaCol("index", IntegerType, nullable = false)) // now NOT NULL + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 1) + assert(errors.head == "`index` type has changed from INT to INT NOT NULL") + } + + test("validateCapturedMetadataColumns - column removed") { + val originMetaCols = Seq(metaCol("index", IntegerType, nullable = true)) // originally present + val currentMetaCols = Array.empty[MetadataColumn] // no metadata columns + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 1) + assert(errors.head == "`index` INT has been removed") + } + + test("validateCapturedMetadataColumns - table doesn't support metadata") { + // table that doesn't implement SupportsMetadataColumns + val table = TestTable("test", Array(col("id", LongType, nullable = true))) + val originMetaCols = Seq(metaCol("index", IntegerType, nullable = false)) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 1) + assert(errors.head == "`index` INT NOT NULL has been removed") + } + + test("validateCapturedMetadataColumns - multiple errors") { + val originMetaCols = Seq( + metaCol("_partition", StringType, nullable = false), + metaCol("index", IntegerType, nullable = false)) // originally present + val currentMetaCols = Array( + metaCol("_partition", IntegerType, nullable = false)) // type changed from StringType + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 2) + assert(errors.exists(e => e.contains("_partition") && e.contains("type has changed"))) + assert(errors.exists(e => e.contains("index") && e.contains("removed"))) + } + + test("validateCapturedMetadataColumns - case insensitive names") { + val originMetaCols = Seq(metaCol("index", IntegerType, nullable = true)) // lowercase + val currentMetaCols = Array(metaCol("INDEX", IntegerType, nullable = true)) // uppercase + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.isEmpty, "Case insensitive comparison should match") + } + + test("validateCapturedMetadataColumns - duplicate metadata columns with different case") { + val originMetaCols = Seq( + metaCol("_partition", StringType, nullable = false), + metaCol("index", IntegerType, nullable = false)) + val currentMetaCols = Array( + metaCol("_partition", StringType, nullable = false), + metaCol("index", IntegerType, nullable = false), + metaCol("INDEX", StringType, nullable = false)) // duplicate with different case + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val e = intercept[AnalysisException] { + V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + } + assert(e.message.contains("Choose another name or rename the existing column")) + } + + test("validateCapturedMetadataColumns - empty metadata columns") { + val originMetaCols = Seq.empty[MetadataColumn] + val currentMetaCols = Array.empty[MetadataColumn] + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.isEmpty, "No metadata columns should produce no errors") + } + + test("validateCapturedMetadataColumns - complex metadata type") { + val structType = StructType(Seq( + StructField("bucket", IntegerType), + StructField("partition", IntegerType))) + val originMetaCols = Seq(metaCol("_partition", structType, nullable = false)) + val currentMetaCols = Array(metaCol("_partition", structType, nullable = false)) + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.isEmpty) + } + + test("validateCapturedMetadataColumns - complex metadata type changed") { + val originStructType = StructType(Seq( + StructField("bucket", IntegerType), // originally IntegerType + StructField("partition", IntegerType))) + val originMetaCols = Seq(metaCol("_partition", originStructType, nullable = false)) + val currentStructType = StructType(Seq( + StructField("bucket", StringType), // changed type + StructField("partition", IntegerType))) + val currentMetaCols = Array(metaCol("_partition", currentStructType, nullable = false)) + val table = TestTableWithMetadataSupport("test", Array.empty, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) + assert(errors.size == 1) + assert(errors.head.contains( + "`_partition` type has changed from STRUCT NOT NULL to " + + "STRUCT NOT NULL")) + } + + test("validateCapturedMetadataColumns - with DataSourceV2Relation") { + val dataCols = Array( + col("id", LongType, nullable = true), + col("name", StringType, nullable = true)) + val originMetaCols = Array( + metaCol("_partition", StringType, nullable = false), + metaCol("index", IntegerType, nullable = false)) + val originTable = TestTableWithMetadataSupport("test", dataCols, originMetaCols) + + val dataAttrs = dataCols.map(c => AttributeReference(c.name, c.dataType, c.nullable)()) + val metadataAttrs = originMetaCols.map(c => MetadataAttribute(c.name, c.dataType, c.isNullable)) + val attrs = dataAttrs ++ metadataAttrs + + val relation = DataSourceV2Relation( + originTable, + attrs.toImmutableArraySeq, + None, + None, + CaseInsensitiveStringMap.empty()) + + val currentMetaCols = Array( + metaCol("_partition", IntegerType, nullable = false), // type changed + metaCol("index", IntegerType, nullable = false)) + val currentTable = TestTableWithMetadataSupport("test", dataCols, currentMetaCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation) + assert(errors.size == 1) + assert(errors.head.contains("`_partition` type has changed")) + } + + test("validateCapturedMetadataColumns - with DataSourceV2Relation no metadata attrs") { + val dataCols = Array( + col("id", LongType, nullable = true), + col("name", StringType, nullable = true)) + val originTable = TestTable("test", dataCols) + + val dataAttrs = dataCols.map(c => AttributeReference(c.name, c.dataType, c.nullable)()) + + val relation = DataSourceV2Relation( + originTable, + dataAttrs.toImmutableArraySeq, + None, + None, + CaseInsensitiveStringMap.empty()) + + val currentTable = TestTable("test", dataCols) + + val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation) + assert(errors.isEmpty) + } + + // simple table without metadata column support + private case class TestTable( + override val name: String, + override val columns: Array[Column]) + extends Table { + override def capabilities: util.Set[TableCapability] = util.Set.of(BATCH_READ) + } + + // simple table implementation with metadata column support + private case class TestTableWithMetadataSupport( + override val name: String, + override val columns: Array[Column], + override val metadataColumns: Array[MetadataColumn] = Array.empty) + extends Table with SupportsMetadataColumns { + override def capabilities: util.Set[TableCapability] = util.Set.of(BATCH_READ) + } + + private case class TestMetadataColumn( + override val name: String, + override val dataType: DataType, + override val isNullable: Boolean) + extends MetadataColumn { + override def comment: String = s"Test metadata column $name" + override def metadataInJSON: String = "{}" + } + + private def validateCapturedColumns(table: Table, originCols: Array[Column]): Seq[String] = { + V2TableUtil.validateCapturedColumns(table, originCols.toImmutableArraySeq) + } + + private def col(name: String, dataType: DataType, nullable: Boolean): Column = { + Column.create(name, dataType, nullable) + } + + private def metaCol( + name: String, + dataType: DataType, + nullable: Boolean): MetadataColumn = { + TestMetadataColumn(name, dataType, nullable) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 27d6eec46b69a..5c7fbcc8edd04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableUnnecessaryBucketedScan} +import org.apache.spark.sql.execution.datasources.v2.V2TableRefreshUtil import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.execution.reuse.ReuseExchangeAndSubquery @@ -203,8 +204,16 @@ class QueryExecution( } } + // there may be delay between analysis and subsequent phases + // therefore, refresh captured table versions to reflect latest data + private val lazyTableVersionsRefreshed = LazyTry { + V2TableRefreshUtil.refreshVersions(commandExecuted) + } + + private[sql] def tableVersionsRefreshed: LogicalPlan = lazyTableVersionsRefreshed.get + private val lazyNormalized = LazyTry { - QueryExecution.normalize(sparkSession, commandExecuted, Some(tracker)) + QueryExecution.normalize(sparkSession, tableVersionsRefreshed, Some(tracker)) } // The plan that has been normalized by custom rules, so that it's more likely to hit cache. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 0c22e13bcb92a..94fc250f4f831 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -27,12 +27,14 @@ import org.apache.spark.sql.QueryTest.withQueryExecutionsCaptured import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, ReplaceTableAsSelect} import org.apache.spark.sql.connector.catalog.{Column, ColumnDefaultValue, DefaultValue, Identifier, InMemoryTableCatalog, TableInfo} +import org.apache.spark.sql.connector.catalog.BasicInMemoryTableCatalog import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, UpdateColumnDefaultValue} import org.apache.spark.sql.connector.expressions.{ApplyTransform, GeneralScalarExpression, LiteralValue, Transform} import org.apache.spark.sql.connector.expressions.filter.{AlwaysFalse, AlwaysTrue} import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.ExplainUtils.stripAQEPlan import org.apache.spark.sql.execution.datasources.v2.{AlterTableExec, CreateTableExec, DataSourceV2Relation, ReplaceTableExec} +import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, CalendarIntervalType, DoubleType, IntegerType, StringType, TimestampType} import org.apache.spark.sql.util.QueryExecutionListener @@ -46,6 +48,7 @@ class DataSourceV2DataFrameSuite override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.ANSI_ENABLED, true) .set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + .set("spark.sql.catalog.testcat.copyOnLoad", "true") .set("spark.sql.catalog.testcat2", classOf[InMemoryTableCatalog].getName) after { @@ -1057,4 +1060,302 @@ class DataSourceV2DataFrameSuite (!compareValue || left.getValue == right.getValue) } } + + test("SPARK-54157: detect table ID change after DataFrame analysis") { + val t = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + withTable(t) { + sql(s"CREATE TABLE $t (id INT, data STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a'), (2, 'b')") + + // create DataFrame and trigger analysis + val df = spark.table(t) + + // capture original table + val originalTable = catalog("testcat").loadTable(ident) + val originalId = originalTable.id() + + // drop and recreate table with same name and schema + sql(s"DROP TABLE $t") + sql(s"CREATE TABLE $t (id INT, data STRING) USING foo") + sql(s"INSERT INTO $t VALUES (3, 'c')") + + // load new table + val newTable = catalog("testcat").loadTable(ident) + val newId = newTable.id() + + // verify IDs are different + assert(originalId != newId) + + // execution should fail with table ID mismatch + checkError( + exception = intercept[AnalysisException] { df.collect() }, + condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.TABLE_ID_MISMATCH", + sqlState = Some("51024"), + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "capturedTableId" -> originalId, + "currentTableId" -> newId)) + } + } + + test("SPARK-54157: detect column removal after DataFrame analysis") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, data STRING, extra STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', 'x')") + + // create DataFrame and trigger analysis + val df = spark.table(t).select($"id", $"data", $"extra") + + // remove column in table + sql(s"ALTER TABLE $t DROP COLUMN extra") + + // execution should fail with column mismatch + checkError( + exception = intercept[AnalysisException] { df.collect() }, + condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "errors" -> "\n- `extra` STRING has been removed")) + } + } + + test("SPARK-54157: detect column addition after DataFrame analysis") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, data STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a')") + + // create DataFrame and trigger analysis + val df = spark.table(t) + + // add columns to table + sql(s"ALTER TABLE $t ADD COLUMN new_col1 INT") + sql(s"ALTER TABLE $t ADD COLUMN new_col2 INT") + + // execution should fail with column mismatch + checkError( + exception = intercept[AnalysisException] { df.collect() }, + condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "errors" -> + """ + |- `new_col1` INT has been added + |- `new_col2` INT has been added""".stripMargin)) + } + } + + test("SPARK-54157: detect multiple change types after DataFrame analysis") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (col1 INT, col2 STRING, col3 BOOLEAN, col4 STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', true, 'x')") + + // create DataFrame and trigger analysis + val df = spark.table(t).select($"col1", $"col2", $"col3", $"col4") + + // make multiple changes in table + sql(s"ALTER TABLE $t DROP COLUMN col4") + sql(s"ALTER TABLE $t ADD COLUMN col5 INT") + + // execution should fail with column mismatch + checkError( + exception = intercept[AnalysisException] { df.collect() }, + condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "errors" -> + """ + |- `col4` STRING has been removed + |- `col5` INT has been added""".stripMargin)) + } + } + + test("SPARK-54157: detect nested struct field changes after DataFrame analysis") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, person STRUCT) USING foo") + sql(s"INSERT INTO $t SELECT 1, named_struct('name', 'Alice', 'age', 30)") + + // create DataFrame and trigger analysis + val df = spark.table(t) + + // add nested field to struct column + sql(s"ALTER TABLE $t ADD COLUMN person.city STRING") + + // execution should fail with column mismatch + checkError( + exception = intercept[AnalysisException] { df.collect() }, + condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "errors" -> + ("\n- `person` type has changed from STRUCT " + + "to STRUCT"))) + } + } + + test("SPARK-54157: detect schema changes in join with same table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, name STRING, value INT) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', 10), (2, 'b', 20)") + + // create first DataFrame + val df1 = spark.table(t) + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) + + // insert more data + sql(s"INSERT INTO $t VALUES (3, 'c', 30)") + + // create second DataFrame with new data + val df2 = spark.table(t) + checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + + // it should be valid to join df1 and df2 + // Spark will refresh versions in joined DataFrame before execution + assert(df1.join(df2, df1("id") === df2("id")).count() == 3) + + // df1 has been executed that must have pinned the version + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) + + // add column and insert more data + sql(s"ALTER TABLE $t ADD COLUMN extra STRING") + sql(s"INSERT INTO $t VALUES (4, 'd', 40, 'x')") + + // create third DataFrame with new data and schema + val df3 = spark.table(t) + checkAnswer(df3, Seq( + Row(1, "a", 10, null), + Row(2, "b", 20, null), + Row(3, "c", 30, null), + Row(4, "d", 40, "x"))) + + // join between df1 and df3 should fail as refreshing versions is not + // sufficient because df1 was resolved with old schema + checkError( + exception = intercept[AnalysisException] { + df1.join(df3, df1("id") === df3("id")).collect() + }, + condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "errors" -> "\n- `extra` STRING has been added")) + + // DataFrame execution before joins must have pinned used versions + // subsequent version refreshes must not be visible in original DataFrames + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) + checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + checkAnswer(df3, Seq( + Row(1, "a", 10, null), + Row(2, "b", 20, null), + Row(3, "c", 30, null), + Row(4, "d", 40, "x"))) + } + } + + test("SPARK-54157: join time travel and current version") { + val t = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + val version = "v1" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, name STRING, value INT) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', 10), (2, 'b', 20)") + + pinTable("testcat", ident, version) + + // insert data + sql(s"INSERT INTO $t VALUES (3, 'c', 30)") + + // create first DataFrame pointing to current version + val df1 = spark.table(t) + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + + // create second DataFrame with time travel + val df2 = spark.sql(s"SELECT * FROM $t VERSION AS OF '$version'") + checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20))) + + // it should be valid to join df1 and df2 despite version mismatch + // as df2 was created using time travel + assert(df1.join(df2, df1("id") === df2("id")).count() == 2) + } + } + + test("SPARK-54157: version is refreshed before cache lookup") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, name STRING, value INT) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', 10), (2, 'b', 20)") + + // create first DataFrame without executing it + val df1 = spark.table(t) + + // insert data + sql(s"INSERT INTO $t VALUES (3, 'c', 30)") + + // create second DataFrame and cache it + val df2 = spark.table(t) + df2.cache() + assertCached(df2) + checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + + // execute first DataFrame that should trigger version refresh + assertCached(df1) + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + } + } + + test("SPARK-54157: replace table as select reading from same table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, name STRING, data STRING, extra INT) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', 'x', 100), (2, 'b', 'y', 200), (3, 'c', 'z', 300)") + + checkAnswer( + spark.table(t), + Seq(Row(1, "a", "x", 100), Row(2, "b", "y", 200), Row(3, "c", "z", 300))) + + // replace table with subset of columns from itself using DataFrame API + // RTAS drops original table before executing query so refresh is special + val df = spark.table(t).select($"id", $"name") + df.writeTo(t).replace() + + // verify table was replaced with only selected columns + checkAnswer( + spark.table(t), + Seq(Row(1, "a"), Row(2, "b"), Row(3, "c"))) + } + } + + test("SPARK-54157: insert overwrite reading from same table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, value INT, category STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 10, 'A'), (2, 20, 'B'), (3, 30, 'A')") + + checkAnswer( + spark.table(t), + Seq(Row(1, 10, "A"), Row(2, 20, "B"), Row(3, 30, "A"))) + + // overwrite with transformed data from same table using DataFrame API + val df = spark.table(t) + .filter($"category" === "A") + .select($"id", ($"value" * 2).as("value"), $"category") + df.writeTo(t).overwrite(lit(true)) + + // verify table was overwritten with transformed data + checkAnswer( + spark.table(t), + Seq(Row(1, 20, "A"), Row(3, 60, "A"))) + } + } + + private def pinTable(catalogName: String, ident: Identifier, version: String): Unit = { + catalog(catalogName) match { + case inMemory: BasicInMemoryTableCatalog => inMemory.pinTable(ident, version) + case _ => fail(s"can't pin $ident in $catalogName") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 6f79e62a3a680..907820e462380 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -965,8 +965,14 @@ class DataSourceV2SQLSuiteV1Filter checkAnswer(sql(s"SELECT * FROM $view"), spark.table("source").select("id")) val oldView = spark.table(view) + assert(spark.sharedState.cacheManager.numCachedEntries == 1) sql(s"REPLACE TABLE $t (a bigint) USING foo") - assert(spark.sharedState.cacheManager.lookupCachedData(oldView).isEmpty) + // it is no longer valid to materialize oldView as underlying + // query execution captured original table before replace + // yet cache invalidation must work correctly + val e = intercept[AnalysisException] { oldView.collect() } + assert(e.message.contains("Table ID has changed")) + assert(spark.sharedState.cacheManager.isEmpty) } } } @@ -3977,6 +3983,12 @@ class V2CatalogSupportBuiltinDataSource extends InMemoryCatalog { ) V1Table(sparkTable) } + + override def loadTable( + ident: Identifier, + writePrivileges: util.Set[TableWritePrivilege]): Table = { + loadTable(ident) + } } class ReadOnlyCatalog extends InMemoryCatalog { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala index 524c0e138721d..9d666b2f0f2f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala @@ -19,7 +19,12 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.Row import org.apache.spark.sql.classic.MergeIntoWriter +import org.apache.spark.sql.connector.catalog.Column +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableInfo import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.StringType class MergeIntoDataFrameSuite extends RowLevelOperationSuiteBase { @@ -971,4 +976,51 @@ class MergeIntoDataFrameSuite extends RowLevelOperationSuiteBase { assert(writer1.schemaEvolutionEnabled) } } + + test("SPARK-54157: version is refreshed when source is V2 table") { + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + // create source table + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("salary", IntegerType), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + sql(s"INSERT INTO $sourceTable VALUES (1, 101, 'support'), (3, 301, 'support')") + + // create source DataFrame without executing it + val sourceDF = spark.table(sourceTable) + + // insert more data into source table + sql(s"INSERT INTO $sourceTable VALUES (4, 401, 'finance')") + + // use source in merge - should refresh and see all data including new insert + sourceDF + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .merge() + + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 101, "support"), // update + Row(2, 200, "software"), // unchanged + Row(3, 301, "support"), // insert + Row(4, 401, "finance"))) // insert + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index f6d2e096ecacc..cd52c75106e17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import java.util + import scala.jdk.CollectionConverters._ import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} @@ -551,17 +553,26 @@ class GlobalTempViewTestSuite extends TempViewTestSuite with SharedSparkSession } class OneTableCatalog extends InMemoryCatalog { + // store table in variable to preserve its ID between load requests + private val table = new InMemoryTable( + "t", + StructType.fromDDL("c1 INT"), + Array.empty, + Map.empty[String, String].asJava) + override def loadTable(ident: Identifier): Table = { if (ident.namespace.isEmpty && ident.name == "t") { - new InMemoryTable( - "t", - StructType.fromDDL("c1 INT"), - Array.empty, - Map.empty[String, String].asJava) + table } else { super.loadTable(ident) } } + + override def loadTable( + ident: Identifier, + writePrivileges: util.Set[TableWritePrivilege]): Table = { + loadTable(ident) + } } class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { From 01b6e734fe4872da8f62497894a4fe9d49c6906b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 14 Nov 2025 10:33:30 -0800 Subject: [PATCH 140/400] [SPARK-54356][SDP] Fix EndToEndAPISuite caused by missing storage root schema ### What changes were proposed in this pull request? Fixes the `EndToEndAPISuite`, which was broken by https://github.com/apache/spark/pull/52999. ### Why are the changes needed? For CI to pass ### Does this PR introduce _any_ user-facing change? No - test-only change. ### How was this patch tested? Ran tests. ### Was this patch authored or co-authored using generative AI tooling? Closes #53069 from sryza/end-to-end-storage-root. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 7599b2ffb28a369d63c6e75cbd616b1e3e156e90) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/connect/pipelines/EndToEndAPISuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/EndToEndAPISuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/EndToEndAPISuite.scala index 55b8a315df570..f674b45bb072d 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/EndToEndAPISuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/EndToEndAPISuite.scala @@ -162,7 +162,7 @@ class EndToEndAPISuite extends PipelineTest with APITest with SparkConnectServer |name: test-pipeline |${spec.catalog.map(catalog => s"""catalog: "$catalog"""").getOrElse("")} |${spec.database.map(database => s"""database: "$database"""").getOrElse("")} - |storage: "${projectDir.resolve("storage").toAbsolutePath}" + |storage: "file://${projectDir.resolve("storage").toAbsolutePath}" |configuration: | "spark.remote": "sc://localhost:$serverPort" |libraries: From 8601aeb391bad930a5fd173bd8d52beb4489466f Mon Sep 17 00:00:00 2001 From: Yuheng Chang Date: Fri, 14 Nov 2025 11:02:13 -0800 Subject: [PATCH 141/400] [SPARK-54348][INFRA] Recover Python unit tests CI by installing `zstandard==0.25.0` ### What changes were proposed in this pull request? In https://github.com/apache/spark/pull/53024#issuecomment-3530340077, PR CI Python unit tests failed due to ``` pyspark.errors.exceptions.base.PySparkImportError: [PACKAGE_NOT_INSTALLED] zstandard >= 0.25.0 must be installed; however, it was not found. ``` This PR add the required dependency to the pre-merge CI. ### Why are the changes needed? Recover Python unit tests CI ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? PR #53024 Python CI back to healthy with this change ### Was this patch authored or co-authored using generative AI tooling? No Closes #53058 from SCHJonathan/jonathan-chang_data/fix-python-ci-dep. Authored-by: Yuheng Chang Signed-off-by: Dongjoon Hyun (cherry picked from commit a916690d6101419297c98a4c42ecf47e24f97f43) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 2 +- .github/workflows/maven_test.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index d0682cc92414e..161164dc5866b 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -361,7 +361,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') || contains(matrix.modules, 'yarn') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'zstandard==0.25.0' python3.11 -m pip list # Run the tests. - name: Run tests diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 7bbfc420e02ab..666a504ad0732 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -175,7 +175,7 @@ jobs: - name: Install Python packages (Python 3.11) if: contains(matrix.modules, 'resource-managers#yarn') || (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' + python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'zstandard==0.25.0' python3.11 -m pip list # Run the tests using script command. # BSD's script command doesn't support -c option, and the usage is different from Linux's one. From 4adea7dddc98f02fb1a50ec4aa3af473109e545f Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Fri, 14 Nov 2025 12:32:37 -0800 Subject: [PATCH 142/400] [SPARK-54172][SQL][FOLLOW-UP] Simplify attribute re-resolution and validate action assignment value resolution for schema evolution ### What changes were proposed in this pull request? Addressed remaining review comments in https://github.com/apache/spark/pull/52866 ### Why are the changes needed? Small code simplification, safety ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #53071 from szehon-ho/merge_schema_evolution_limit_follow. Authored-by: Szehon Ho Signed-off-by: Dongjoon Hyun (cherry picked from commit dc5bb9641b6e6fb8f55b1c524b3a21d4b5063283) Signed-off-by: Dongjoon Hyun --- .../catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala | 4 +--- .../spark/sql/catalyst/plans/logical/v2Commands.scala | 6 +++++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala index f317a2efddbe4..ea0883f7928f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala @@ -48,9 +48,7 @@ object ResolveMergeIntoSchemaEvolution extends Rule[LogicalPlan] { val newTarget = performSchemaEvolution(r, referencedSourceSchema, changes) val oldTargetOutput = m.targetTable.output val newTargetOutput = newTarget.output - val attributeMapping = oldTargetOutput.map( - oldAttr => (oldAttr, newTargetOutput.find(_.name == oldAttr.name).getOrElse(oldAttr)) - ) + val attributeMapping = oldTargetOutput.zip(newTargetOutput) newTarget -> attributeMapping } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index ea9f7c91cf005..c9d6fb61ba69f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -928,9 +928,13 @@ case class MergeIntoTable( }.flatten val sourcePaths = MergeIntoTable.extractAllFieldPaths(sourceTable.schema) + // Only allow unresolved assignment keys to be candidates for schema evolution + // if they are directly assigned from source fields, ie UPDATE SET new = source.new assignments.forall { assignment => assignment.resolved || - sourcePaths.exists { path => MergeIntoTable.isEqual(assignment, path) } + (assignment.value.resolved && sourcePaths.exists { + path => MergeIntoTable.isEqual(assignment, path) + }) } } } From d8dc6e36cab561fa4194903a97bba40a484f45ab Mon Sep 17 00:00:00 2001 From: ibenchhida Date: Fri, 14 Nov 2025 12:57:59 -0800 Subject: [PATCH 143/400] [SPARK-53337][UI] XSS: Ensure the application name in historypage get escaped ### What changes were proposed in this pull request? In history Server main dashboard, the application name need to be escaped. ### Why are the changes needed? Not escaped app name could lead to XSS Exemple: cat << EOF > script.py from pyspark.sql import SparkSession if __name__ == "__main__": spark = SparkSession \ .builder \ .appName("") \ .getOrCreate() print(spark.range(1000 * 1000 * 1000).count()) spark.stop() EOF ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests already exist, it was manually tested. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52851 from Chhida/Fix_issue_master. Authored-by: ibenchhida Signed-off-by: Dongjoon Hyun (cherry picked from commit e739b7d349dbd41b78d49bc8086cab63db057e14) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/ui/static/historypage.js | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 8961140a40190..d847efa2f7ff0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -27,6 +27,16 @@ var appLimit = -1; function setAppLimit(val) { appLimit = val; } +/* escape XSS */ +function escapeHtml(text) { + if (typeof text !== 'string') return text; + return text + .replace(/&/g, "&") + .replace(//g, ">") + .replace(/"/g, """) + .replace(/'/g, "'"); +} /* eslint-enable no-unused-vars*/ function makeIdNumeric(id) { @@ -151,7 +161,7 @@ $(document).ready(function() { attempt["durationMillisec"] = attempt["duration"]; attempt["duration"] = formatDuration(attempt["duration"]); attempt["id"] = id; - attempt["name"] = name; + attempt["name"] = escapeHtml(name); attempt["version"] = version; attempt["attemptUrl"] = uiRoot + "/history/" + id + "/" + (attempt.hasOwnProperty("attemptId") ? attempt["attemptId"] + "/" : "") + "jobs/"; From c96b9d7da5399c5b4e5f8bde6327f31ee30fe9df Mon Sep 17 00:00:00 2001 From: fanyue-xia Date: Fri, 14 Nov 2025 13:01:41 -0800 Subject: [PATCH 144/400] [SPARK-50906][SQL] Fix Avro nullability check for reordered struct fields ### What changes were proposed in this pull request? When Catalyst and Avro schemas have different field orders, the nullability validation was incorrectly using positional indexing instead of name-based field mapping. The fix uses avroIndices(i) to map Catalyst field positions to Avro field positions when checking nullability and generating error messages. Added two tests to verify: 1. Reordered fields with nullable target succeed 2. Reordered fields with non-nullable target fail with correct field name ### Why are the changes needed? fix a bug, as explained above ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? U/T ### Was this patch authored or co-authored using generative AI tooling? 'Generated-by: Claude Code version 2.0.37' Closes #53041 from fanyue-xia/fix-avro-nullability-reordered-fields. Lead-authored-by: fanyue-xia Co-authored-by: Gengliang Wang Signed-off-by: Gengliang Wang (cherry picked from commit cf513611bf7956091ca17a2956cd4fd52a1c25d5) Signed-off-by: Gengliang Wang --- .../org/apache/spark/sql/avro/AvroSuite.scala | 55 +++++++++++++++++++ .../spark/sql/avro/AvroSerializer.scala | 6 +- 2 files changed, 58 insertions(+), 3 deletions(-) diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 0b3823ca16160..b0f510f3257ef 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1423,6 +1423,61 @@ abstract class AvroSuite } } + test("to_avro with reordered fields and nullable target succeeds") { + // Test that when Catalyst and Avro field orders differ, null values + // are correctly validated against the mapped Avro field's nullability + val avroSchema = """{ + "type": "record", + "name": "ReorderedRecord", + "fields": [ + {"name": "a", "type": ["null", "string"]}, + {"name": "b", "type": "string"} + ] + }""" + + // Catalyst has fields in order [b, a], Avro has [a, b] + // Pass null for 'a' which is nullable in Avro - should succeed + val df = Seq(("B", null.asInstanceOf[String])).toDF("b", "a") + .select(struct($"b", $"a").as("s")) + val result = df.select(avro.functions.to_avro($"s", avroSchema).as("avro")) + + // Should succeed without throwing AVRO_CANNOT_WRITE_NULL_FIELD + val collected = result.collect() + assert(collected.length == 1) + + // Verify data correctness by round-tripping through from_avro + val roundTrip = result.select(avro.functions.from_avro($"avro", avroSchema).as("s")) + // final field order should be [a, b] as per avro schema + checkAnswer(roundTrip, Row(Row(null, "B"))) + } + + test("to_avro with reordered fields fails with correct field name") { + // Test that when Catalyst and Avro field orders differ and we try to write + // null to a non-nullable field, the error message references the correct field name + val avroSchema = """{ + "type": "record", + "name": "ReorderedRecord", + "fields": [ + {"name": "a", "type": ["null", "string"]}, + {"name": "b", "type": "string"} + ] + }""" + + // Catalyst has fields in order [b, a], Avro has [a, b] + // Pass null for 'b' which is non-nullable in Avro - should fail with correct field name 'b' + val df = Seq((null.asInstanceOf[String], "A")).toDF("b", "a") + .select(struct($"b", $"a").as("s")) + + checkError( + exception = intercept[SparkRuntimeException] { + df.select(avro.functions.to_avro($"s", avroSchema)).collect() + }, + condition = "AVRO_CANNOT_WRITE_NULL_FIELD", + parameters = Map( + "name" -> "`b`", + "dataType" -> "\"string\"")) + } + test("support user provided avro schema for writing nullable fixed type") { withTempPath { tempDir => val avroSchema = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala index 402bab666948d..f3c6e168d2f04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala @@ -291,12 +291,12 @@ private[sql] class AvroSerializer( var i = 0 while (i < numFields) { if (row.isNullAt(i)) { - if (!isSchemaNullable(i)) { + if (!isSchemaNullable(avroIndices(i))) { throw new SparkRuntimeException( errorClass = "AVRO_CANNOT_WRITE_NULL_FIELD", messageParameters = Map( - "name" -> toSQLId(avroFields.get(i).name), - "dataType" -> avroFields.get(i).schema().toString)) + "name" -> toSQLId(avroFields.get(avroIndices(i)).name), + "dataType" -> avroFields.get(avroIndices(i)).schema().toString)) } result.put(avroIndices(i), null) } else { From 724ef245baa51b7f7092fd43244b58f6e10212d5 Mon Sep 17 00:00:00 2001 From: Yuheng Chang Date: Fri, 14 Nov 2025 16:44:25 -0800 Subject: [PATCH 145/400] [SPARK-54020] Support `spark.sql(...)` Python API inside query functions for Spark Declarative Pipeline ### What changes were proposed in this pull request? This PR adds support for `spark.sql(...)` Python API inside query functions for Spark Declarative Pipelines. Users can now use `spark.sql(...)` to define query functions, and dependencies are correctly tracked. **Example usage:** ```python dp.materialized_view() def source(): return spark.sql("SELECT * FROM RANGE(5)") dp.materialized_view() def target(): return spark.sql("SELECT * FROM source") ``` This PR also adds restrictions on the set of SQL commands users can execute. Unsupported commands (e.g., `spark.sql("CREATE TABLE ...")`) inside query functions will raise an error. **Implementation details:** 1. Added `PipelineAnalysisContext` to Spark Connect's user context extensions, enabling the server to identify requests originating from Spark Declarative Pipelines and apply appropriate restrictions. 2. The `flow_name` field in `PipelineAnalysisContext` determines execution behavior: - **Inside query functions** (`flow_name` is set): Spark Connect server treats `spark.sql()` as a no-op and returns the raw logical plan to SDP for deferred analysis as part of the Dataflow Graph. - **Outside query functions** (`flow_name` is empty): Spark Connect server eagerly executes the command, but only SDP-allowlisted commands are permitted. ### Why are the changes needed? `spark.sql(...)` is a common and intuitive pattern for users who are more familiar with SQL to define query functions. Supporting this API improves usability and allows SQL-first developers to work more naturally with Spark Declarative Pipelines. ### Does this PR introduce _any_ user-facing change? Yes. Previously, `spark.sql(...)` inside query functions was not supported and users would see an `ATTEMPT_ANALYSIS_IN_PIPELINE_QUERY_FUNCTION` exception. This PR lifts that restriction. ### How was this patch tested? New test cases in `PythonPipelineSuite` unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #53024 from SCHJonathan/jonathan-chang_data/spark-sql. Authored-by: Yuheng Chang Signed-off-by: Dongjoon Hyun (cherry picked from commit cc72c647b6f92023dfb2d1d0b25198fa37573f00) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 6 + .../add_pipeline_analysis_context.py | 48 ++++ .../pyspark/pipelines/block_connect_access.py | 46 +++- python/pyspark/pipelines/cli.py | 17 +- .../spark_connect_graph_element_registry.py | 9 +- .../test_add_pipeline_analysis_context.py | 100 ++++++++ python/pyspark/pipelines/tests/test_cli.py | 15 +- .../pyspark/pipelines/tests/test_init_cli.py | 2 +- python/pyspark/sql/connect/client/core.py | 59 ++++- .../sql/connect/proto/pipelines_pb2.py | 8 +- .../sql/connect/proto/pipelines_pb2.pyi | 16 ++ .../protobuf/spark/connect/pipelines.proto | 2 + .../connect/pipelines/PipelinesHandler.scala | 51 +++- .../connect/planner/SparkConnectPlanner.scala | 38 ++- .../pipelines/PythonPipelineSuite.scala | 235 ++++++++++++++++-- ...SparkDeclarativePipelinesServerSuite.scala | 224 +++++++++-------- 16 files changed, 732 insertions(+), 144 deletions(-) create mode 100644 python/pyspark/pipelines/add_pipeline_analysis_context.py create mode 100644 python/pyspark/pipelines/tests/test_add_pipeline_analysis_context.py diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index ca8e953a22460..98c61aa33f2e2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -6935,6 +6935,12 @@ ], "sqlState" : "0A000" }, + "UNSUPPORTED_PIPELINE_SPARK_SQL_COMMAND" : { + "message" : [ + "'' is not supported in spark.sql(\"...\") API in Spark Declarative Pipeline." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_SAVE_MODE" : { "message" : [ "The save mode is not supported for:" diff --git a/python/pyspark/pipelines/add_pipeline_analysis_context.py b/python/pyspark/pipelines/add_pipeline_analysis_context.py new file mode 100644 index 0000000000000..6ecabdf43b072 --- /dev/null +++ b/python/pyspark/pipelines/add_pipeline_analysis_context.py @@ -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. +# +from contextlib import contextmanager +from typing import Generator, Optional +from pyspark.sql import SparkSession + +from typing import Any, cast + + +@contextmanager +def add_pipeline_analysis_context( + spark: SparkSession, dataflow_graph_id: str, flow_name: Optional[str] +) -> Generator[None, None, None]: + """ + Context manager that add PipelineAnalysisContext extension to the user context + used for pipeline specific analysis. + """ + extension_id = None + # Cast because mypy seems to think `spark` is a function, not an object. + # Likely related to SPARK-47544. + client = cast(Any, spark).client + try: + import pyspark.sql.connect.proto as pb2 + from google.protobuf import any_pb2 + + analysis_context = pb2.PipelineAnalysisContext( + dataflow_graph_id=dataflow_graph_id, flow_name=flow_name + ) + extension = any_pb2.Any() + extension.Pack(analysis_context) + extension_id = client.add_threadlocal_user_context_extension(extension) + yield + finally: + client.remove_user_context_extension(extension_id) diff --git a/python/pyspark/pipelines/block_connect_access.py b/python/pyspark/pipelines/block_connect_access.py index c5dacbbc2c5cb..696d0e39b005d 100644 --- a/python/pyspark/pipelines/block_connect_access.py +++ b/python/pyspark/pipelines/block_connect_access.py @@ -15,7 +15,7 @@ # limitations under the License. # from contextlib import contextmanager -from typing import Callable, Generator, NoReturn +from typing import Any, Callable, Generator from pyspark.errors import PySparkException from pyspark.sql.connect.proto.base_pb2_grpc import SparkConnectServiceStub @@ -24,6 +24,27 @@ BLOCKED_RPC_NAMES = ["AnalyzePlan", "ExecutePlan"] +def _is_sql_command_request(rpc_name: str, args: tuple) -> bool: + """ + Check if the RPC call is a spark.sql() command (ExecutePlan with sql_command). + + :param rpc_name: Name of the RPC being called + :param args: Arguments passed to the RPC + :return: True if this is an ExecutePlan request with a sql_command + """ + if rpc_name != "ExecutePlan" or len(args) == 0: + return False + + request = args[0] + if not hasattr(request, "plan"): + return False + plan = request.plan + if not plan.HasField("command"): + return False + command = plan.command + return command.HasField("sql_command") + + @contextmanager def block_spark_connect_execution_and_analysis() -> Generator[None, None, None]: """ @@ -38,16 +59,23 @@ def block_spark_connect_execution_and_analysis() -> Generator[None, None, None]: # Define a new __getattribute__ method that blocks RPC calls def blocked_getattr(self: SparkConnectServiceStub, name: str) -> Callable: - if name not in BLOCKED_RPC_NAMES: - return original_getattr(self, name) + original_method = original_getattr(self, name) - def blocked_method(*args: object, **kwargs: object) -> NoReturn: - raise PySparkException( - errorClass="ATTEMPT_ANALYSIS_IN_PIPELINE_QUERY_FUNCTION", - messageParameters={}, - ) + def intercepted_method(*args: object, **kwargs: object) -> Any: + # Allow all RPCs that are not AnalyzePlan or ExecutePlan + if name not in BLOCKED_RPC_NAMES: + return original_method(*args, **kwargs) + # Allow spark.sql() commands (ExecutePlan with sql_command) + elif _is_sql_command_request(name, args): + return original_method(*args, **kwargs) + # Block all other AnalyzePlan and ExecutePlan calls + else: + raise PySparkException( + errorClass="ATTEMPT_ANALYSIS_IN_PIPELINE_QUERY_FUNCTION", + messageParameters={}, + ) - return blocked_method + return intercepted_method try: # Apply our custom __getattribute__ method diff --git a/python/pyspark/pipelines/cli.py b/python/pyspark/pipelines/cli.py index ca198f1c3aff3..3ba0bb58fe946 100644 --- a/python/pyspark/pipelines/cli.py +++ b/python/pyspark/pipelines/cli.py @@ -49,6 +49,8 @@ handle_pipeline_events, ) +from pyspark.pipelines.add_pipeline_analysis_context import add_pipeline_analysis_context + PIPELINE_SPEC_FILE_NAMES = ["pipeline.yaml", "pipeline.yml"] @@ -216,7 +218,11 @@ def validate_str_dict(d: Mapping[str, str], field_name: str) -> Mapping[str, str def register_definitions( - spec_path: Path, registry: GraphElementRegistry, spec: PipelineSpec + spec_path: Path, + registry: GraphElementRegistry, + spec: PipelineSpec, + spark: SparkSession, + dataflow_graph_id: str, ) -> None: """Register the graph element definitions in the pipeline spec with the given registry. - Looks for Python files matching the glob patterns in the spec and imports them. @@ -245,8 +251,11 @@ def register_definitions( assert ( module_spec.loader is not None ), f"Module spec has no loader for {file}" - with block_session_mutations(): - module_spec.loader.exec_module(module) + with add_pipeline_analysis_context( + spark=spark, dataflow_graph_id=dataflow_graph_id, flow_name=None + ): + with block_session_mutations(): + module_spec.loader.exec_module(module) elif file.suffix == ".sql": log_with_curr_timestamp(f"Registering SQL file {file}...") with file.open("r") as f: @@ -324,7 +333,7 @@ def run( log_with_curr_timestamp("Registering graph elements...") registry = SparkConnectGraphElementRegistry(spark, dataflow_graph_id) - register_definitions(spec_path, registry, spec) + register_definitions(spec_path, registry, spec, spark, dataflow_graph_id) log_with_curr_timestamp("Starting run...") result_iter = start_run( diff --git a/python/pyspark/pipelines/spark_connect_graph_element_registry.py b/python/pyspark/pipelines/spark_connect_graph_element_registry.py index e8a8561c3e749..b8d297fced3fb 100644 --- a/python/pyspark/pipelines/spark_connect_graph_element_registry.py +++ b/python/pyspark/pipelines/spark_connect_graph_element_registry.py @@ -35,6 +35,7 @@ from pyspark.sql.types import StructType from typing import Any, cast import pyspark.sql.connect.proto as pb2 +from pyspark.pipelines.add_pipeline_analysis_context import add_pipeline_analysis_context class SparkConnectGraphElementRegistry(GraphElementRegistry): @@ -43,6 +44,7 @@ class SparkConnectGraphElementRegistry(GraphElementRegistry): def __init__(self, spark: SparkSession, dataflow_graph_id: str) -> None: # Cast because mypy seems to think `spark`` is a function, not an object. Likely related to # SPARK-47544. + self._spark = spark self._client = cast(Any, spark).client self._dataflow_graph_id = dataflow_graph_id @@ -110,8 +112,11 @@ def register_output(self, output: Output) -> None: self._client.execute_command(command) def register_flow(self, flow: Flow) -> None: - with block_spark_connect_execution_and_analysis(): - df = flow.func() + with add_pipeline_analysis_context( + spark=self._spark, dataflow_graph_id=self._dataflow_graph_id, flow_name=flow.name + ): + with block_spark_connect_execution_and_analysis(): + df = flow.func() relation = cast(ConnectDataFrame, df)._plan.plan(self._client) relation_flow_details = pb2.PipelineCommand.DefineFlow.WriteRelationFlowDetails( diff --git a/python/pyspark/pipelines/tests/test_add_pipeline_analysis_context.py b/python/pyspark/pipelines/tests/test_add_pipeline_analysis_context.py new file mode 100644 index 0000000000000..57c5da22d4601 --- /dev/null +++ b/python/pyspark/pipelines/tests/test_add_pipeline_analysis_context.py @@ -0,0 +1,100 @@ +# +# 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. +# +import unittest + +from pyspark.testing.connectutils import ( + ReusedConnectTestCase, + should_test_connect, + connect_requirement_message, +) + +if should_test_connect: + from pyspark.pipelines.add_pipeline_analysis_context import add_pipeline_analysis_context + + +@unittest.skipIf(not should_test_connect, connect_requirement_message) +class AddPipelineAnalysisContextTests(ReusedConnectTestCase): + def test_add_pipeline_analysis_context_with_flow_name(self): + with add_pipeline_analysis_context(self.spark, "test_dataflow_graph_id", "test_flow_name"): + import pyspark.sql.connect.proto as pb2 + + thread_local_extensions = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions), 1) + # Extension is stored as (id, extension), unpack the extension + _extension_id, extension = thread_local_extensions[0] + context = pb2.PipelineAnalysisContext() + extension.Unpack(context) + self.assertEqual(context.dataflow_graph_id, "test_dataflow_graph_id") + self.assertEqual(context.flow_name, "test_flow_name") + thread_local_extensions_after = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions_after), 0) + + def test_add_pipeline_analysis_context_without_flow_name(self): + with add_pipeline_analysis_context(self.spark, "test_dataflow_graph_id", None): + import pyspark.sql.connect.proto as pb2 + + thread_local_extensions = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions), 1) + # Extension is stored as (id, extension), unpack the extension + _extension_id, extension = thread_local_extensions[0] + context = pb2.PipelineAnalysisContext() + extension.Unpack(context) + self.assertEqual(context.dataflow_graph_id, "test_dataflow_graph_id") + # Empty string means no flow name + self.assertEqual(context.flow_name, "") + thread_local_extensions_after = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions_after), 0) + + def test_nested_add_pipeline_analysis_context(self): + import pyspark.sql.connect.proto as pb2 + + with add_pipeline_analysis_context(self.spark, "test_dataflow_graph_id_1", flow_name=None): + with add_pipeline_analysis_context( + self.spark, "test_dataflow_graph_id_2", flow_name="test_flow_name" + ): + thread_local_extensions = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions), 2) + # Extension is stored as (id, extension), unpack the extensions + _, extension_1 = thread_local_extensions[0] + context_1 = pb2.PipelineAnalysisContext() + extension_1.Unpack(context_1) + self.assertEqual(context_1.dataflow_graph_id, "test_dataflow_graph_id_1") + self.assertEqual(context_1.flow_name, "") + _, extension_2 = thread_local_extensions[1] + context_2 = pb2.PipelineAnalysisContext() + extension_2.Unpack(context_2) + self.assertEqual(context_2.dataflow_graph_id, "test_dataflow_graph_id_2") + self.assertEqual(context_2.flow_name, "test_flow_name") + thread_local_extensions_after_1 = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions_after_1), 1) + _, extension_3 = thread_local_extensions_after_1[0] + context_3 = pb2.PipelineAnalysisContext() + extension_3.Unpack(context_3) + self.assertEqual(context_3.dataflow_graph_id, "test_dataflow_graph_id_1") + self.assertEqual(context_3.flow_name, "") + thread_local_extensions_after_2 = self.spark.client.thread_local.user_context_extensions + self.assertEqual(len(thread_local_extensions_after_2), 0) + + +if __name__ == "__main__": + try: + import xmlrunner # type: ignore + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/pipelines/tests/test_cli.py b/python/pyspark/pipelines/tests/test_cli.py index e8445e63d439d..ff3022fa29663 100644 --- a/python/pyspark/pipelines/tests/test_cli.py +++ b/python/pyspark/pipelines/tests/test_cli.py @@ -22,6 +22,7 @@ from pyspark.errors import PySparkException from pyspark.testing.connectutils import ( + ReusedConnectTestCase, should_test_connect, connect_requirement_message, ) @@ -45,7 +46,7 @@ not should_test_connect or not have_yaml, connect_requirement_message or yaml_requirement_message, ) -class CLIUtilityTests(unittest.TestCase): +class CLIUtilityTests(ReusedConnectTestCase): def test_load_pipeline_spec(self): with tempfile.NamedTemporaryFile(mode="w") as tmpfile: tmpfile.write( @@ -294,7 +295,9 @@ def mv2(): ) registry = LocalGraphElementRegistry() - register_definitions(outer_dir / "pipeline.yaml", registry, spec) + register_definitions( + outer_dir / "pipeline.yaml", registry, spec, self.spark, "test_graph_id" + ) self.assertEqual(len(registry.outputs), 1) self.assertEqual(registry.outputs[0].name, "mv1") @@ -315,7 +318,9 @@ def test_register_definitions_file_raises_error(self): registry = LocalGraphElementRegistry() with self.assertRaises(RuntimeError) as context: - register_definitions(outer_dir / "pipeline.yml", registry, spec) + register_definitions( + outer_dir / "pipeline.yml", registry, spec, self.spark, "test_graph_id" + ) self.assertIn("This is a test exception", str(context.exception)) def test_register_definitions_unsupported_file_extension_matches_glob(self): @@ -334,7 +339,7 @@ def test_register_definitions_unsupported_file_extension_matches_glob(self): registry = LocalGraphElementRegistry() with self.assertRaises(PySparkException) as context: - register_definitions(outer_dir, registry, spec) + register_definitions(outer_dir, registry, spec, self.spark, "test_graph_id") self.assertEqual( context.exception.getCondition(), "PIPELINE_UNSUPPORTED_DEFINITIONS_FILE_EXTENSION" ) @@ -382,6 +387,8 @@ def test_python_import_current_directory(self): configuration={}, libraries=[LibrariesGlob(include="defs.py")], ), + self.spark, + "test_graph_id", ) def test_full_refresh_all_conflicts_with_full_refresh(self): diff --git a/python/pyspark/pipelines/tests/test_init_cli.py b/python/pyspark/pipelines/tests/test_init_cli.py index 43c553eddc387..e51bab6a4a691 100644 --- a/python/pyspark/pipelines/tests/test_init_cli.py +++ b/python/pyspark/pipelines/tests/test_init_cli.py @@ -60,7 +60,7 @@ def test_init(self): self.assertTrue((Path.cwd() / "pipeline-storage").exists()) registry = LocalGraphElementRegistry() - register_definitions(spec_path, registry, spec) + register_definitions(spec_path, registry, spec, self.spark, "test_graph_id") self.assertEqual(len(registry.outputs), 1) self.assertEqual(registry.outputs[0].name, "example_python_materialized_view") self.assertEqual(len(registry.flows), 1) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 44fa31474f590..b4da47191d838 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -727,6 +727,9 @@ def __init__( # cleanup ml cache if possible atexit.register(self._cleanup_ml_cache) + self.global_user_context_extensions: List[Tuple[str, any_pb2.Any]] = [] + self.global_user_context_extensions_lock = threading.Lock() + @property def _stub(self) -> grpc_lib.SparkConnectServiceStub: if self.is_closed: @@ -1258,6 +1261,24 @@ def token(self) -> Optional[str]: """ return self._builder.token + def _update_request_with_user_context_extensions( + self, + req: Union[ + pb2.AnalyzePlanRequest, + pb2.ConfigRequest, + pb2.ExecutePlanRequest, + pb2.FetchErrorDetailsRequest, + pb2.InterruptRequest, + ], + ) -> None: + with self.global_user_context_extensions_lock: + for _, extension in self.global_user_context_extensions: + req.user_context.extensions.append(extension) + if not hasattr(self.thread_local, "user_context_extensions"): + return + for _, extension in self.thread_local.user_context_extensions: + req.user_context.extensions.append(extension) + def _execute_plan_request_with_metadata( self, operation_id: Optional[str] = None ) -> pb2.ExecutePlanRequest: @@ -1288,6 +1309,7 @@ def _execute_plan_request_with_metadata( messageParameters={"arg_name": "operation_id", "origin": str(ve)}, ) req.operation_id = operation_id + self._update_request_with_user_context_extensions(req) return req def _analyze_plan_request_with_metadata(self) -> pb2.AnalyzePlanRequest: @@ -1298,6 +1320,7 @@ def _analyze_plan_request_with_metadata(self) -> pb2.AnalyzePlanRequest: req.client_type = self._builder.userAgent if self._user_id: req.user_context.user_id = self._user_id + self._update_request_with_user_context_extensions(req) return req def _analyze(self, method: str, **kwargs: Any) -> AnalyzeResult: @@ -1712,6 +1735,7 @@ def _config_request_with_metadata(self) -> pb2.ConfigRequest: req.client_type = self._builder.userAgent if self._user_id: req.user_context.user_id = self._user_id + self._update_request_with_user_context_extensions(req) return req def get_configs(self, *keys: str) -> Tuple[Optional[str], ...]: @@ -1788,6 +1812,7 @@ def _interrupt_request( ) if self._user_id: req.user_context.user_id = self._user_id + self._update_request_with_user_context_extensions(req) return req def interrupt_all(self) -> Optional[List[str]]: @@ -1886,6 +1911,38 @@ def _throw_if_invalid_tag(self, tag: str) -> None: messageParameters={"arg_name": "Spark Connect tag", "arg_value": tag}, ) + def add_threadlocal_user_context_extension(self, extension: any_pb2.Any) -> str: + if not hasattr(self.thread_local, "user_context_extensions"): + self.thread_local.user_context_extensions = list() + extension_id = "threadlocal_" + str(uuid.uuid4()) + self.thread_local.user_context_extensions.append((extension_id, extension)) + return extension_id + + def add_global_user_context_extension(self, extension: any_pb2.Any) -> str: + extension_id = "global_" + str(uuid.uuid4()) + with self.global_user_context_extensions_lock: + self.global_user_context_extensions.append((extension_id, extension)) + return extension_id + + def remove_user_context_extension(self, extension_id: str) -> None: + if extension_id.find("threadlocal_") == 0: + if not hasattr(self.thread_local, "user_context_extensions"): + return + self.thread_local.user_context_extensions = list( + filter(lambda ex: ex[0] != extension_id, self.thread_local.user_context_extensions) + ) + elif extension_id.find("global_") == 0: + with self.global_user_context_extensions_lock: + self.global_user_context_extensions = list( + filter(lambda ex: ex[0] != extension_id, self.global_user_context_extensions) + ) + + def clear_user_context_extensions(self) -> None: + if hasattr(self.thread_local, "user_context_extensions"): + self.thread_local.user_context_extensions = list() + with self.global_user_context_extensions_lock: + self.global_user_context_extensions = list() + def _handle_error(self, error: Exception) -> NoReturn: """ Handle errors that occur during RPC calls. @@ -1926,7 +1983,7 @@ def _fetch_enriched_error(self, info: "ErrorInfo") -> Optional[pb2.FetchErrorDet req.client_observed_server_side_session_id = self._server_session_id if self._user_id: req.user_context.user_id = self._user_id - + self._update_request_with_user_context_extensions(req) try: return self._stub.FetchErrorDetails(req, metadata=self._builder.metadata()) except grpc.RpcError: diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py index 0eb77c84b5b57..7a30def861d29 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.py +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py @@ -42,7 +42,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\xed"\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12R\n\rdefine_output\x18\x02 \x01(\x0b\x32+.spark.connect.PipelineCommand.DefineOutputH\x00R\x0c\x64\x65\x66ineOutput\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x12\xa1\x01\n*get_query_function_execution_signal_stream\x18\x07 \x01(\x0b\x32\x44.spark.connect.PipelineCommand.GetQueryFunctionExecutionSignalStreamH\x00R%getQueryFunctionExecutionSignalStream\x12\x88\x01\n!define_flow_query_function_result\x18\x08 \x01(\x0b\x32<.spark.connect.PipelineCommand.DefineFlowQueryFunctionResultH\x00R\x1d\x64\x65\x66ineFlowQueryFunctionResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xb4\x02\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x92\n\n\x0c\x44\x65\x66ineOutput\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12$\n\x0boutput_name\x18\x02 \x01(\tH\x02R\noutputName\x88\x01\x01\x12?\n\x0boutput_type\x18\x03 \x01(\x0e\x32\x19.spark.connect.OutputTypeH\x03R\noutputType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x04R\x07\x63omment\x88\x01\x01\x12X\n\x14source_code_location\x18\x05 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12_\n\rtable_details\x18\x06 \x01(\x0b\x32\x38.spark.connect.PipelineCommand.DefineOutput.TableDetailsH\x00R\x0ctableDetails\x12\\\n\x0csink_details\x18\x07 \x01(\x0b\x32\x37.spark.connect.PipelineCommand.DefineOutput.SinkDetailsH\x00R\x0bsinkDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xc0\x03\n\x0cTableDetails\x12x\n\x10table_properties\x18\x01 \x03(\x0b\x32M.spark.connect.PipelineCommand.DefineOutput.TableDetails.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x02 \x03(\tR\rpartitionCols\x12\x1b\n\x06\x66ormat\x18\x03 \x01(\tH\x01R\x06\x66ormat\x88\x01\x01\x12\x43\n\x10schema_data_type\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x0eschemaDataType\x12%\n\rschema_string\x18\x05 \x01(\tH\x00R\x0cschemaString\x12-\n\x12\x63lustering_columns\x18\x06 \x03(\tR\x11\x63lusteringColumns\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x08\n\x06schemaB\t\n\x07_format\x1a\xd1\x01\n\x0bSinkDetails\x12^\n\x07options\x18\x01 \x03(\x0b\x32\x44.spark.connect.PipelineCommand.DefineOutput.SinkDetails.OptionsEntryR\x07options\x12\x1b\n\x06\x66ormat\x18\x02 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0e\n\x0c_output_nameB\x0e\n\x0c_output_typeB\n\n\x08_commentB\x17\n\x15_source_code_location\x1a\xff\x06\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x03R\x11targetDatasetName\x88\x01\x01\x12Q\n\x08sql_conf\x18\x04 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12 \n\tclient_id\x18\x05 \x01(\tH\x04R\x08\x63lientId\x88\x01\x01\x12X\n\x14source_code_location\x18\x06 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12x\n\x15relation_flow_details\x18\x07 \x01(\x0b\x32\x42.spark.connect.PipelineCommand.DefineFlow.WriteRelationFlowDetailsH\x00R\x13relationFlowDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x17\n\x04once\x18\x08 \x01(\x08H\x06R\x04once\x88\x01\x01\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x61\n\x18WriteRelationFlowDetails\x12\x38\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x08relation\x88\x01\x01\x42\x0b\n\t_relation\x1a:\n\x08Response\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x42\x0c\n\n_flow_nameB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0c\n\n_client_idB\x17\n\x15_source_code_locationB\x07\n\x05_once\x1a\xc2\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x12\x1d\n\x07storage\x18\x06 \x01(\tH\x03R\x07storage\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dryB\n\n\x08_storage\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_text\x1a\x9e\x01\n%GetQueryFunctionExecutionSignalStream\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tclient_id\x18\x02 \x01(\tH\x01R\x08\x63lientId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_client_id\x1a\xdd\x01\n\x1d\x44\x65\x66ineFlowQueryFunctionResult\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x12/\n\x11\x64\x61taflow_graph_id\x18\x02 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x38\n\x08relation\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x02R\x08relation\x88\x01\x01\x42\x0c\n\n_flow_nameB\x14\n\x12_dataflow_graph_idB\x0b\n\t_relationB\x0e\n\x0c\x63ommand_type"\xf0\x05\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x12k\n\x14\x64\x65\x66ine_output_result\x18\x02 \x01(\x0b\x32\x37.spark.connect.PipelineCommandResult.DefineOutputResultH\x00R\x12\x64\x65\x66ineOutputResult\x12\x65\n\x12\x64\x65\x66ine_flow_result\x18\x03 \x01(\x0b\x32\x35.spark.connect.PipelineCommandResult.DefineFlowResultH\x00R\x10\x64\x65\x66ineFlowResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x85\x01\n\x12\x44\x65\x66ineOutputResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifier\x1a\x83\x01\n\x10\x44\x65\x66ineFlowResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifierB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message"\xf1\x01\n\x12SourceCodeLocation\x12 \n\tfile_name\x18\x01 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12$\n\x0bline_number\x18\x02 \x01(\x05H\x01R\nlineNumber\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x03 \x01(\tH\x02R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x0c\n\n_file_nameB\x0e\n\x0c_line_numberB\x12\n\x10_definition_path"E\n$PipelineQueryFunctionExecutionSignal\x12\x1d\n\nflow_names\x18\x01 \x03(\tR\tflowNames"\xd7\x01\n\x17PipelineAnalysisContext\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x02 \x01(\tH\x01R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x14\n\x12_dataflow_graph_idB\x12\n\x10_definition_path*i\n\nOutputType\x12\x1b\n\x17OUTPUT_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x12\x08\n\x04SINK\x10\x04\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\xed"\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12R\n\rdefine_output\x18\x02 \x01(\x0b\x32+.spark.connect.PipelineCommand.DefineOutputH\x00R\x0c\x64\x65\x66ineOutput\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x12\xa1\x01\n*get_query_function_execution_signal_stream\x18\x07 \x01(\x0b\x32\x44.spark.connect.PipelineCommand.GetQueryFunctionExecutionSignalStreamH\x00R%getQueryFunctionExecutionSignalStream\x12\x88\x01\n!define_flow_query_function_result\x18\x08 \x01(\x0b\x32<.spark.connect.PipelineCommand.DefineFlowQueryFunctionResultH\x00R\x1d\x64\x65\x66ineFlowQueryFunctionResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xb4\x02\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x92\n\n\x0c\x44\x65\x66ineOutput\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12$\n\x0boutput_name\x18\x02 \x01(\tH\x02R\noutputName\x88\x01\x01\x12?\n\x0boutput_type\x18\x03 \x01(\x0e\x32\x19.spark.connect.OutputTypeH\x03R\noutputType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x04R\x07\x63omment\x88\x01\x01\x12X\n\x14source_code_location\x18\x05 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12_\n\rtable_details\x18\x06 \x01(\x0b\x32\x38.spark.connect.PipelineCommand.DefineOutput.TableDetailsH\x00R\x0ctableDetails\x12\\\n\x0csink_details\x18\x07 \x01(\x0b\x32\x37.spark.connect.PipelineCommand.DefineOutput.SinkDetailsH\x00R\x0bsinkDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\xc0\x03\n\x0cTableDetails\x12x\n\x10table_properties\x18\x01 \x03(\x0b\x32M.spark.connect.PipelineCommand.DefineOutput.TableDetails.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x02 \x03(\tR\rpartitionCols\x12\x1b\n\x06\x66ormat\x18\x03 \x01(\tH\x01R\x06\x66ormat\x88\x01\x01\x12\x43\n\x10schema_data_type\x18\x04 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x0eschemaDataType\x12%\n\rschema_string\x18\x05 \x01(\tH\x00R\x0cschemaString\x12-\n\x12\x63lustering_columns\x18\x06 \x03(\tR\x11\x63lusteringColumns\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x08\n\x06schemaB\t\n\x07_format\x1a\xd1\x01\n\x0bSinkDetails\x12^\n\x07options\x18\x01 \x03(\x0b\x32\x44.spark.connect.PipelineCommand.DefineOutput.SinkDetails.OptionsEntryR\x07options\x12\x1b\n\x06\x66ormat\x18\x02 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0e\n\x0c_output_nameB\x0e\n\x0c_output_typeB\n\n\x08_commentB\x17\n\x15_source_code_location\x1a\xff\x06\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x03R\x11targetDatasetName\x88\x01\x01\x12Q\n\x08sql_conf\x18\x04 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12 \n\tclient_id\x18\x05 \x01(\tH\x04R\x08\x63lientId\x88\x01\x01\x12X\n\x14source_code_location\x18\x06 \x01(\x0b\x32!.spark.connect.SourceCodeLocationH\x05R\x12sourceCodeLocation\x88\x01\x01\x12x\n\x15relation_flow_details\x18\x07 \x01(\x0b\x32\x42.spark.connect.PipelineCommand.DefineFlow.WriteRelationFlowDetailsH\x00R\x13relationFlowDetails\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x17\n\x04once\x18\x08 \x01(\x08H\x06R\x04once\x88\x01\x01\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x61\n\x18WriteRelationFlowDetails\x12\x38\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x08relation\x88\x01\x01\x42\x0b\n\t_relation\x1a:\n\x08Response\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x42\x0c\n\n_flow_nameB\t\n\x07\x64\x65tailsB\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0c\n\n_client_idB\x17\n\x15_source_code_locationB\x07\n\x05_once\x1a\xc2\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x12\x1d\n\x07storage\x18\x06 \x01(\tH\x03R\x07storage\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dryB\n\n\x08_storage\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_text\x1a\x9e\x01\n%GetQueryFunctionExecutionSignalStream\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tclient_id\x18\x02 \x01(\tH\x01R\x08\x63lientId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_client_id\x1a\xdd\x01\n\x1d\x44\x65\x66ineFlowQueryFunctionResult\x12 \n\tflow_name\x18\x01 \x01(\tH\x00R\x08\x66lowName\x88\x01\x01\x12/\n\x11\x64\x61taflow_graph_id\x18\x02 \x01(\tH\x01R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x38\n\x08relation\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x02R\x08relation\x88\x01\x01\x42\x0c\n\n_flow_nameB\x14\n\x12_dataflow_graph_idB\x0b\n\t_relationB\x0e\n\x0c\x63ommand_type"\xf0\x05\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x12k\n\x14\x64\x65\x66ine_output_result\x18\x02 \x01(\x0b\x32\x37.spark.connect.PipelineCommandResult.DefineOutputResultH\x00R\x12\x64\x65\x66ineOutputResult\x12\x65\n\x12\x64\x65\x66ine_flow_result\x18\x03 \x01(\x0b\x32\x35.spark.connect.PipelineCommandResult.DefineFlowResultH\x00R\x10\x64\x65\x66ineFlowResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\x85\x01\n\x12\x44\x65\x66ineOutputResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifier\x1a\x83\x01\n\x10\x44\x65\x66ineFlowResult\x12W\n\x13resolved_identifier\x18\x01 \x01(\x0b\x32!.spark.connect.ResolvedIdentifierH\x00R\x12resolvedIdentifier\x88\x01\x01\x42\x16\n\x14_resolved_identifierB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message"\xf1\x01\n\x12SourceCodeLocation\x12 \n\tfile_name\x18\x01 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12$\n\x0bline_number\x18\x02 \x01(\x05H\x01R\nlineNumber\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x03 \x01(\tH\x02R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x0c\n\n_file_nameB\x0e\n\x0c_line_numberB\x12\n\x10_definition_path"E\n$PipelineQueryFunctionExecutionSignal\x12\x1d\n\nflow_names\x18\x01 \x03(\tR\tflowNames"\x87\x02\n\x17PipelineAnalysisContext\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12,\n\x0f\x64\x65\x66inition_path\x18\x02 \x01(\tH\x01R\x0e\x64\x65\x66initionPath\x88\x01\x01\x12 \n\tflow_name\x18\x03 \x01(\tH\x02R\x08\x66lowName\x88\x01\x01\x12\x33\n\textension\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\textensionB\x14\n\x12_dataflow_graph_idB\x12\n\x10_definition_pathB\x0c\n\n_flow_name*i\n\nOutputType\x12\x1b\n\x17OUTPUT_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x12\x08\n\x04SINK\x10\x04\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -69,8 +69,8 @@ ]._serialized_options = b"8\001" _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._loaded_options = None _globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_options = b"8\001" - _globals["_OUTPUTTYPE"]._serialized_start = 6139 - _globals["_OUTPUTTYPE"]._serialized_end = 6244 + _globals["_OUTPUTTYPE"]._serialized_start = 6187 + _globals["_OUTPUTTYPE"]._serialized_end = 6292 _globals["_PIPELINECOMMAND"]._serialized_start = 195 _globals["_PIPELINECOMMAND"]._serialized_end = 4656 _globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_start = 1129 @@ -126,5 +126,5 @@ _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_start = 5850 _globals["_PIPELINEQUERYFUNCTIONEXECUTIONSIGNAL"]._serialized_end = 5919 _globals["_PIPELINEANALYSISCONTEXT"]._serialized_start = 5922 - _globals["_PIPELINEANALYSISCONTEXT"]._serialized_end = 6137 + _globals["_PIPELINEANALYSISCONTEXT"]._serialized_end = 6185 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi index e0768a1f6baeb..39a1e29ae7dde 100644 --- a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi +++ b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi @@ -1499,11 +1499,14 @@ class PipelineAnalysisContext(google.protobuf.message.Message): DATAFLOW_GRAPH_ID_FIELD_NUMBER: builtins.int DEFINITION_PATH_FIELD_NUMBER: builtins.int + FLOW_NAME_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int dataflow_graph_id: builtins.str """Unique identifier of the dataflow graph associated with this pipeline.""" definition_path: builtins.str """The path of the top-level pipeline file determined at runtime during pipeline initialization.""" + flow_name: builtins.str + """The name of the Flow involved in this analysis""" @property def extension( self, @@ -1516,6 +1519,7 @@ class PipelineAnalysisContext(google.protobuf.message.Message): *, dataflow_graph_id: builtins.str | None = ..., definition_path: builtins.str | None = ..., + flow_name: builtins.str | None = ..., extension: collections.abc.Iterable[google.protobuf.any_pb2.Any] | None = ..., ) -> None: ... def HasField( @@ -1525,10 +1529,14 @@ class PipelineAnalysisContext(google.protobuf.message.Message): b"_dataflow_graph_id", "_definition_path", b"_definition_path", + "_flow_name", + b"_flow_name", "dataflow_graph_id", b"dataflow_graph_id", "definition_path", b"definition_path", + "flow_name", + b"flow_name", ], ) -> builtins.bool: ... def ClearField( @@ -1538,12 +1546,16 @@ class PipelineAnalysisContext(google.protobuf.message.Message): b"_dataflow_graph_id", "_definition_path", b"_definition_path", + "_flow_name", + b"_flow_name", "dataflow_graph_id", b"dataflow_graph_id", "definition_path", b"definition_path", "extension", b"extension", + "flow_name", + b"flow_name", ], ) -> None: ... @typing.overload @@ -1554,5 +1566,9 @@ class PipelineAnalysisContext(google.protobuf.message.Message): def WhichOneof( self, oneof_group: typing_extensions.Literal["_definition_path", b"_definition_path"] ) -> typing_extensions.Literal["definition_path"] | None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_flow_name", b"_flow_name"] + ) -> typing_extensions.Literal["flow_name"] | None: ... global___PipelineAnalysisContext = PipelineAnalysisContext diff --git a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto index a92e24fda9154..0874c2d10ec5c 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto @@ -299,6 +299,8 @@ message PipelineAnalysisContext { optional string dataflow_graph_id = 1; // The path of the top-level pipeline file determined at runtime during pipeline initialization. optional string definition_path = 2; + // The name of the Flow involved in this analysis + optional string flow_name = 3; // Reserved field for protocol extensions. repeated google.protobuf.Any extension = 999; diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala index 1a3b0d2231c62..4c60e0f70ff4c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.connect.pipelines +import scala.collection.Seq import scala.jdk.CollectionConverters._ import scala.util.Using @@ -27,9 +28,10 @@ import org.apache.spark.connect.proto.{ExecutePlanResponse, PipelineCommandResul import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Command, CreateNamespace, CreateTable, CreateTableAsSelect, CreateView, DescribeRelation, DropView, InsertIntoStatement, LogicalPlan, RenameTable, ShowColumns, ShowCreateTable, ShowFunctions, ShowTableProperties, ShowTables, ShowViews} import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.connect.service.SessionHolder +import org.apache.spark.sql.execution.command.{ShowCatalogsCommand, ShowNamespacesCommand} import org.apache.spark.sql.pipelines.Language.Python import org.apache.spark.sql.pipelines.common.RunState.{CANCELED, FAILED} import org.apache.spark.sql.pipelines.graph.{AllTables, FlowAnalysis, GraphIdentifierManager, GraphRegistrationContext, IdentifierHelper, NoTables, PipelineUpdateContextImpl, QueryContext, QueryOrigin, QueryOriginType, Sink, SinkImpl, SomeTables, SqlGraphRegistrationContext, Table, TableFilter, TemporaryView, UnresolvedFlow} @@ -129,6 +131,50 @@ private[connect] object PipelinesHandler extends Logging { } } + /** + * Block SQL commands that have side effects or modify data. + * + * Pipeline definitions should be declarative and side-effect free. This prevents users from + * inadvertently modifying catalogs, creating tables, or performing other stateful operations + * outside the pipeline API boundary during pipeline registration or analysis. + * + * This is a best-effort approach: we block known problematic commands while allowing a curated + * set of read-only operations (e.g., SHOW, DESCRIBE). + */ + def blockUnsupportedSqlCommand(queryPlan: LogicalPlan): Unit = { + val allowlistedCommands = Set( + classOf[DescribeRelation], + classOf[ShowTables], + classOf[ShowTableProperties], + classOf[ShowNamespacesCommand], + classOf[ShowColumns], + classOf[ShowFunctions], + classOf[ShowViews], + classOf[ShowCatalogsCommand], + classOf[ShowCreateTable]) + val isSqlCommandExplicitlyAllowlisted = allowlistedCommands.exists(_.isInstance(queryPlan)) + val isUnsupportedSqlPlan = if (isSqlCommandExplicitlyAllowlisted) { + false + } else { + // Disable all [[Command]] except the ones that are explicitly allowlisted + // in "allowlistedCommands". + queryPlan.isInstanceOf[Command] || + // Following commands are not subclasses of [[Command]] but have side effects. + queryPlan.isInstanceOf[CreateTableAsSelect] || + queryPlan.isInstanceOf[CreateTable] || + queryPlan.isInstanceOf[CreateView] || + queryPlan.isInstanceOf[InsertIntoStatement] || + queryPlan.isInstanceOf[RenameTable] || + queryPlan.isInstanceOf[CreateNamespace] || + queryPlan.isInstanceOf[DropView] + } + if (isUnsupportedSqlPlan) { + throw new AnalysisException( + "UNSUPPORTED_PIPELINE_SPARK_SQL_COMMAND", + Map("command" -> queryPlan.getClass.getSimpleName)) + } + } + private def createDataflowGraph( cmd: proto.PipelineCommand.CreateDataflowGraph, sessionHolder: SessionHolder): String = { @@ -148,6 +194,9 @@ private[connect] object PipelinesHandler extends Logging { val defaultSqlConf = cmd.getSqlConfMap.asScala.toMap + sessionHolder.session.catalog.setCurrentCatalog(defaultCatalog) + sessionHolder.session.catalog.setCurrentDatabase(defaultDatabase) + sessionHolder.dataflowGraphRegistry.createDataflowGraph( defaultCatalog = defaultCatalog, defaultDatabase = defaultDatabase, diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 8bc33c41b3a30..644784fa3db6c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -21,11 +21,12 @@ import java.util.{HashMap, Properties, UUID} import scala.collection.mutable import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag import scala.util.Try import scala.util.control.NonFatal import com.google.common.collect.Lists -import com.google.protobuf.{Any => ProtoAny, ByteString} +import com.google.protobuf.{Any => ProtoAny, ByteString, Message} import io.grpc.{Context, Status, StatusRuntimeException} import io.grpc.stub.StreamObserver @@ -33,7 +34,7 @@ import org.apache.spark.{SparkClassNotFoundException, SparkEnv, SparkException, import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction} import org.apache.spark.connect.proto -import org.apache.spark.connect.proto.{CheckpointCommand, CreateResourceProfileCommand, ExecutePlanResponse, SqlCommand, StreamingForeachFunction, StreamingQueryCommand, StreamingQueryCommandResult, StreamingQueryInstanceId, StreamingQueryManagerCommand, StreamingQueryManagerCommandResult, WriteStreamOperationStart, WriteStreamOperationStartResult} +import org.apache.spark.connect.proto.{CheckpointCommand, CreateResourceProfileCommand, ExecutePlanResponse, PipelineAnalysisContext, SqlCommand, StreamingForeachFunction, StreamingQueryCommand, StreamingQueryCommandResult, StreamingQueryInstanceId, StreamingQueryManagerCommand, StreamingQueryManagerCommandResult, WriteStreamOperationStart, WriteStreamOperationStartResult} import org.apache.spark.connect.proto.ExecutePlanResponse.SqlCommandResult import org.apache.spark.connect.proto.Parse.ParseFormat import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult.StreamingQueryInstance @@ -2941,10 +2942,28 @@ class SparkConnectPlanner( .build()) } + private def getExtensionList[T <: Message: ClassTag]( + extensions: mutable.Buffer[ProtoAny]): Seq[T] = { + val cls = implicitly[ClassTag[T]].runtimeClass + .asInstanceOf[Class[_ <: Message]] + extensions.collect { + case any if any.is(cls) => any.unpack(cls).asInstanceOf[T] + }.toSeq + } + private def handleSqlCommand( command: SqlCommand, responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { val tracker = executeHolder.eventsManager.createQueryPlanningTracker() + val userContextExtensions = executeHolder.request.getUserContext.getExtensionsList.asScala + val pipelineAnalysisContextList = { + getExtensionList[PipelineAnalysisContext](userContextExtensions) + } + val hasPipelineAnalysisContext = pipelineAnalysisContextList.nonEmpty + val insidePipelineFlowFunction = pipelineAnalysisContextList.exists(_.hasFlowName) + // To avoid explicit handling of the result on the client, we build the expected input + // of the relation on the server. The client has to simply forward the result. + val result = SqlCommandResult.newBuilder() val relation = if (command.hasInput) { command.getInput @@ -2964,6 +2983,18 @@ class SparkConnectPlanner( .build() } + // Block unsupported SQL commands if the request comes from Spark Declarative Pipelines. + if (hasPipelineAnalysisContext) { + PipelinesHandler.blockUnsupportedSqlCommand(queryPlan = transformRelation(relation)) + } + + // If the spark.sql() is called inside a pipeline flow function, we don't need to execute + // the SQL command and defer the actual analysis and execution to the flow function. + if (insidePipelineFlowFunction) { + result.setRelation(relation) + return + } + val df = relation.getRelTypeCase match { case proto.Relation.RelTypeCase.SQL => executeSQL(relation.getSql, tracker) @@ -2982,9 +3013,6 @@ class SparkConnectPlanner( case _ => Seq.empty } - // To avoid explicit handling of the result on the client, we build the expected input - // of the relation on the server. The client has to simply forward the result. - val result = SqlCommandResult.newBuilder() // Only filled when isCommand val metrics = ExecutePlanResponse.Metrics.newBuilder() if (isCommand || isSqlScript) { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala index 1a72d112aa2ef..1850241f07026 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala @@ -50,7 +50,7 @@ class PythonPipelineSuite def buildGraph(pythonText: String): DataflowGraph = { assume(PythonTestDepsChecker.isConnectDepsAvailable) - val indentedPythonText = pythonText.linesIterator.map(" " + _).mkString("\n") + val indentedPythonText = pythonText.linesIterator.map(" " + _).mkString("\n") // create a unique identifier to allow identifying the session and dataflow graph val customSessionIdentifier = UUID.randomUUID().toString val pythonCode = @@ -64,6 +64,9 @@ class PythonPipelineSuite |from pyspark.pipelines.graph_element_registry import ( | graph_element_registration_context, |) + |from pyspark.pipelines.add_pipeline_analysis_context import ( + | add_pipeline_analysis_context + |) | |spark = SparkSession.builder \\ | .remote("sc://localhost:$serverPort") \\ @@ -79,7 +82,10 @@ class PythonPipelineSuite |) | |registry = SparkConnectGraphElementRegistry(spark, dataflow_graph_id) - |with graph_element_registration_context(registry): + |with add_pipeline_analysis_context( + | spark=spark, dataflow_graph_id=dataflow_graph_id, flow_name=None + |): + | with graph_element_registration_context(registry): |$indentedPythonText |""".stripMargin @@ -143,7 +149,7 @@ class PythonPipelineSuite QueryOrigin( language = Option(Python()), filePath = Option(""), - line = Option(28), + line = Option(34), objectName = Option("spark_catalog.default.table1"), objectType = Option(QueryOriginType.Flow.toString))), errorChecker = ex => @@ -195,7 +201,7 @@ class PythonPipelineSuite QueryOrigin( language = Option(Python()), filePath = Option(""), - line = Option(34), + line = Option(40), objectName = Option("spark_catalog.default.mv2"), objectType = Option(QueryOriginType.Flow.toString))), expectedEventLevel = EventLevel.INFO) @@ -209,7 +215,7 @@ class PythonPipelineSuite QueryOrigin( language = Option(Python()), filePath = Option(""), - line = Option(38), + line = Option(44), objectName = Option("spark_catalog.default.mv"), objectType = Option(QueryOriginType.Flow.toString))), expectedEventLevel = EventLevel.INFO) @@ -227,7 +233,7 @@ class PythonPipelineSuite QueryOrigin( language = Option(Python()), filePath = Option(""), - line = Option(28), + line = Option(34), objectName = Option("spark_catalog.default.table1"), objectType = Option(QueryOriginType.Flow.toString))), expectedEventLevel = EventLevel.INFO) @@ -241,7 +247,7 @@ class PythonPipelineSuite QueryOrigin( language = Option(Python()), filePath = Option(""), - line = Option(43), + line = Option(49), objectName = Option("spark_catalog.default.standalone_flow1"), objectType = Option(QueryOriginType.Flow.toString))), expectedEventLevel = EventLevel.INFO) @@ -334,21 +340,35 @@ class PythonPipelineSuite |@dp.table |def b(): | return spark.readStream.table("src") + | + |@dp.materialized_view + |def c(): + | return spark.sql("SELECT * FROM src") + | + |@dp.table + |def d(): + | return spark.sql("SELECT * FROM STREAM src") |""".stripMargin).resolve().validate() assert( graph.table.keySet == Set( graphIdentifier("src"), graphIdentifier("a"), - graphIdentifier("b"))) - Seq("a", "b").foreach { flowName => + graphIdentifier("b"), + graphIdentifier("c"), + graphIdentifier("d"))) + Seq("a", "b", "c").foreach { flowName => // dependency is properly tracked assert(graph.resolvedFlow(graphIdentifier(flowName)).inputs == Set(graphIdentifier("src"))) } val (streamingFlows, batchFlows) = graph.resolvedFlows.partition(_.df.isStreaming) - assert(batchFlows.map(_.identifier) == Seq(graphIdentifier("src"), graphIdentifier("a"))) - assert(streamingFlows.map(_.identifier) == Seq(graphIdentifier("b"))) + assert( + batchFlows.map(_.identifier) == Seq( + graphIdentifier("src"), + graphIdentifier("a"), + graphIdentifier("c"))) + assert(streamingFlows.map(_.identifier) == Seq(graphIdentifier("b"), graphIdentifier("d"))) } test("referencing external datasets") { @@ -365,18 +385,33 @@ class PythonPipelineSuite |@dp.table |def c(): | return spark.readStream.table("spark_catalog.default.src") + | + |@dp.materialized_view + |def d(): + | return spark.sql("SELECT * FROM spark_catalog.default.src") + | + |@dp.table + |def e(): + | return spark.sql("SELECT * FROM STREAM spark_catalog.default.src") |""".stripMargin).resolve().validate() assert( graph.tables.map(_.identifier).toSet == Set( graphIdentifier("a"), graphIdentifier("b"), - graphIdentifier("c"))) + graphIdentifier("c"), + graphIdentifier("d"), + graphIdentifier("e"))) // dependency is not tracked assert(graph.resolvedFlows.forall(_.inputs.isEmpty)) val (streamingFlows, batchFlows) = graph.resolvedFlows.partition(_.df.isStreaming) - assert(batchFlows.map(_.identifier).toSet == Set(graphIdentifier("a"), graphIdentifier("b"))) - assert(streamingFlows.map(_.identifier) == Seq(graphIdentifier("c"))) + assert( + batchFlows.map(_.identifier).toSet == Set( + graphIdentifier("a"), + graphIdentifier("b"), + graphIdentifier("d"))) + assert( + streamingFlows.map(_.identifier).toSet == Set(graphIdentifier("c"), graphIdentifier("e"))) } test("referencing internal datasets failed") { @@ -392,9 +427,17 @@ class PythonPipelineSuite |@dp.table |def c(): | return spark.readStream.table("src") + | + |@dp.materialized_view + |def d(): + | return spark.sql("SELECT * FROM src") + | + |@dp.table + |def e(): + | return spark.sql("SELECT * FROM STREAM src") |""".stripMargin).resolve() - assert(graph.resolutionFailedFlows.size == 3) + assert(graph.resolutionFailedFlows.size == 5) graph.resolutionFailedFlows.foreach { flow => assert(flow.failure.head.getMessage.contains("[TABLE_OR_VIEW_NOT_FOUND]")) assert(flow.failure.head.getMessage.contains("`src`")) @@ -414,12 +457,94 @@ class PythonPipelineSuite |@dp.materialized_view |def c(): | return spark.readStream.table("spark_catalog.default.src") + | + |@dp.materialized_view + |def d(): + | return spark.sql("SELECT * FROM spark_catalog.default.src") + | + |@dp.table + |def e(): + | return spark.sql("SELECT * FROM STREAM spark_catalog.default.src") |""".stripMargin).resolve() + assert(graph.resolutionFailedFlows.size == 5) graph.resolutionFailedFlows.foreach { flow => - assert(flow.failure.head.getMessage.contains("[TABLE_OR_VIEW_NOT_FOUND] The table or view")) + assert(flow.failure.head.getMessage.contains("[TABLE_OR_VIEW_NOT_FOUND]")) + assert(flow.failure.head.getMessage.contains("`spark_catalog`.`default`.`src`")) } } + test("reading external datasets outside query function works") { + sql("CREATE TABLE spark_catalog.default.src AS SELECT * FROM RANGE(5)") + val graph = buildGraph(s""" + |spark_sql_df = spark.sql("SELECT * FROM spark_catalog.default.src") + |read_table_df = spark.read.table("spark_catalog.default.src") + | + |@dp.materialized_view + |def mv_from_spark_sql_df(): + | return spark_sql_df + | + |@dp.materialized_view + |def mv_from_read_table_df(): + | return read_table_df + |""".stripMargin).resolve().validate() + + assert( + graph.resolvedFlows.map(_.identifier).toSet == Set( + graphIdentifier("mv_from_spark_sql_df"), + graphIdentifier("mv_from_read_table_df"))) + assert(graph.resolvedFlows.forall(_.inputs.isEmpty)) + assert(graph.resolvedFlows.forall(!_.df.isStreaming)) + } + + test( + "reading internal datasets outside query function that don't trigger " + + "eager analysis or execution") { + val graph = buildGraph(""" + |@dp.materialized_view + |def src(): + | return spark.range(5) + | + |read_table_df = spark.read.table("src") + | + |@dp.materialized_view + |def mv_from_read_table_df(): + | return read_table_df + | + |""".stripMargin).resolve().validate() + assert( + graph.resolvedFlows.map(_.identifier).toSet == Set( + graphIdentifier("mv_from_read_table_df"), + graphIdentifier("src"))) + assert(graph.resolvedFlows.forall(!_.df.isStreaming)) + assert( + graph + .resolvedFlow(graphIdentifier("mv_from_read_table_df")) + .inputs + .contains(graphIdentifier("src"))) + } + + gridTest( + "reading internal datasets outside query function that trigger " + + "eager analysis or execution will fail")( + Seq("""spark.sql("SELECT * FROM src")""", """spark.read.table("src").collect()""")) { + command => + val ex = intercept[RuntimeException] { + buildGraph(s""" + |@dp.materialized_view + |def src(): + | return spark.range(5) + | + |spark_sql_df = $command + | + |@dp.materialized_view + |def mv_from_spark_sql_df(): + | return spark_sql_df + |""".stripMargin) + } + assert(ex.getMessage.contains("TABLE_OR_VIEW_NOT_FOUND")) + assert(ex.getMessage.contains("`src`")) + } + test("create dataset with the same name will fail") { assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[AnalysisException] { @@ -902,4 +1027,82 @@ class PythonPipelineSuite s"Table should have no transforms, but got: ${stTransforms.mkString(", ")}") } } + + // List of unsupported SQL commands that should result in a failure. + private val unsupportedSqlCommandList: Seq[String] = Seq( + "SET CATALOG some_catalog", + "USE SCHEMA some_schema", + "SET `test_conf` = `true`", + "CREATE TABLE some_table (id INT)", + "CREATE VIEW some_view AS SELECT * FROM some_table", + "INSERT INTO some_table VALUES (1)", + "ALTER TABLE some_table RENAME TO some_new_table", + "CREATE NAMESPACE some_namespace", + "DROP VIEW some_view", + "CREATE MATERIALIZED VIEW some_view AS SELECT * FROM some_table", + "CREATE STREAMING TABLE some_table AS SELECT * FROM some_table") + + gridTest("Unsupported SQL command outside query function should result in a failure")( + unsupportedSqlCommandList) { unsupportedSqlCommand => + val ex = intercept[RuntimeException] { + buildGraph(s""" + |spark.sql("$unsupportedSqlCommand") + | + |@dp.materialized_view() + |def mv(): + | return spark.range(5) + |""".stripMargin) + } + assert(ex.getMessage.contains("UNSUPPORTED_PIPELINE_SPARK_SQL_COMMAND")) + } + + gridTest("Unsupported SQL command inside query function should result in a failure")( + unsupportedSqlCommandList) { unsupportedSqlCommand => + val ex = intercept[RuntimeException] { + buildGraph(s""" + |@dp.materialized_view() + |def mv(): + | spark.sql("$unsupportedSqlCommand") + | return spark.range(5) + |""".stripMargin) + } + assert(ex.getMessage.contains("UNSUPPORTED_PIPELINE_SPARK_SQL_COMMAND")) + } + + // List of supported SQL commands that should work. + val supportedSqlCommandList: Seq[String] = Seq( + "DESCRIBE TABLE spark_catalog.default.src", + "SHOW TABLES", + "SHOW TBLPROPERTIES spark_catalog.default.src", + "SHOW NAMESPACES", + "SHOW COLUMNS FROM spark_catalog.default.src", + "SHOW FUNCTIONS", + "SHOW VIEWS", + "SHOW CATALOGS", + "SHOW CREATE TABLE spark_catalog.default.src", + "SELECT * FROM RANGE(5)", + "SELECT * FROM spark_catalog.default.src") + + gridTest("Supported SQL command outside query function should work")(supportedSqlCommandList) { + supportedSqlCommand => + sql("CREATE TABLE spark_catalog.default.src AS SELECT * FROM RANGE(5)") + buildGraph(s""" + |spark.sql("$supportedSqlCommand") + | + |@dp.materialized_view() + |def mv(): + | return spark.range(5) + |""".stripMargin) + } + + gridTest("Supported SQL command inside query function should work")(supportedSqlCommandList) { + supportedSqlCommand => + sql("CREATE TABLE spark_catalog.default.src AS SELECT * FROM RANGE(5)") + buildGraph(s""" + |@dp.materialized_view() + |def mv(): + | spark.sql("$supportedSqlCommand") + | return spark.range(5) + |""".stripMargin) + } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala index ab60462e87351..c9551646385c2 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala @@ -51,6 +51,32 @@ class SparkDeclarativePipelinesServerSuite } } + test( + "create dataflow graph set session catalog and database to pipeline " + + "default catalog and database") { + withRawBlockingStub { implicit stub => + // Use default spark_catalog and create a test database + sql("CREATE DATABASE IF NOT EXISTS test_db") + try { + val graphId = sendPlan( + buildCreateDataflowGraphPlan( + proto.PipelineCommand.CreateDataflowGraph + .newBuilder() + .setDefaultCatalog("spark_catalog") + .setDefaultDatabase("test_db") + .build())).getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId + val definition = + getDefaultSessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(graphId) + assert(definition.defaultCatalog == "spark_catalog") + assert(definition.defaultDatabase == "test_db") + assert(getDefaultSessionHolder.session.catalog.currentCatalog() == "spark_catalog") + assert(getDefaultSessionHolder.session.catalog.currentDatabase == "test_db") + } finally { + sql("DROP DATABASE IF EXISTS test_db") + } + } + } + test("Define a flow for a graph that does not exist") { val ex = intercept[Exception] { withRawBlockingStub { implicit stub => @@ -515,8 +541,7 @@ class SparkDeclarativePipelinesServerSuite name: String, datasetType: OutputType, datasetName: String, - defaultCatalog: String = "", - defaultDatabase: String = "", + defaultDatabase: String, expectedResolvedDatasetName: String, expectedResolvedCatalog: String, expectedResolvedNamespace: Seq[String]) @@ -526,6 +551,7 @@ class SparkDeclarativePipelinesServerSuite name = "TEMPORARY_VIEW", datasetType = OutputType.TEMPORARY_VIEW, datasetName = "tv", + defaultDatabase = "default", expectedResolvedDatasetName = "tv", expectedResolvedCatalog = "", expectedResolvedNamespace = Seq.empty), @@ -533,6 +559,7 @@ class SparkDeclarativePipelinesServerSuite name = "TABLE", datasetType = OutputType.TABLE, datasetName = "`tb`", + defaultDatabase = "default", expectedResolvedDatasetName = "tb", expectedResolvedCatalog = "spark_catalog", expectedResolvedNamespace = Seq("default")), @@ -540,6 +567,7 @@ class SparkDeclarativePipelinesServerSuite name = "MV", datasetType = OutputType.MATERIALIZED_VIEW, datasetName = "mv", + defaultDatabase = "default", expectedResolvedDatasetName = "mv", expectedResolvedCatalog = "spark_catalog", expectedResolvedNamespace = Seq("default"))).map(tc => tc.name -> tc).toMap @@ -549,7 +577,6 @@ class SparkDeclarativePipelinesServerSuite name = "TEMPORARY_VIEW", datasetType = OutputType.TEMPORARY_VIEW, datasetName = "tv", - defaultCatalog = "custom_catalog", defaultDatabase = "custom_db", expectedResolvedDatasetName = "tv", expectedResolvedCatalog = "", @@ -558,19 +585,17 @@ class SparkDeclarativePipelinesServerSuite name = "TABLE", datasetType = OutputType.TABLE, datasetName = "`tb`", - defaultCatalog = "`my_catalog`", defaultDatabase = "`my_db`", expectedResolvedDatasetName = "tb", - expectedResolvedCatalog = "`my_catalog`", + expectedResolvedCatalog = "spark_catalog", expectedResolvedNamespace = Seq("`my_db`")), DefineOutputTestCase( name = "MV", datasetType = OutputType.MATERIALIZED_VIEW, datasetName = "mv", - defaultCatalog = "another_catalog", defaultDatabase = "another_db", expectedResolvedDatasetName = "mv", - expectedResolvedCatalog = "another_catalog", + expectedResolvedCatalog = "spark_catalog", expectedResolvedNamespace = Seq("another_db"))) .map(tc => tc.name -> tc) .toMap @@ -604,40 +629,45 @@ class SparkDeclarativePipelinesServerSuite } } - namedGridTest("DefineOutput returns resolved data name for custom catalog/schema")( + namedGridTest("DefineOutput returns resolved data name for custom schema")( defineDatasetCustomTests) { testCase => withRawBlockingStub { implicit stub => - // Build and send the CreateDataflowGraph command with custom catalog/db - val graphId = sendPlan( - buildCreateDataflowGraphPlan( - proto.PipelineCommand.CreateDataflowGraph - .newBuilder() - .setDefaultCatalog(testCase.defaultCatalog) - .setDefaultDatabase(testCase.defaultDatabase) - .build())).getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId + sql(s"CREATE DATABASE IF NOT EXISTS spark_catalog.${testCase.defaultDatabase}") + try { + // Build and send the CreateDataflowGraph command with custom catalog/db + val graphId = sendPlan( + buildCreateDataflowGraphPlan( + proto.PipelineCommand.CreateDataflowGraph + .newBuilder() + .setDefaultCatalog("spark_catalog") + .setDefaultDatabase(testCase.defaultDatabase) + .build())).getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId - assert(graphId.nonEmpty) + assert(graphId.nonEmpty) - // Build DefineOutput with the created graphId and dataset info - val defineDataset = DefineOutput - .newBuilder() - .setDataflowGraphId(graphId) - .setOutputName(testCase.datasetName) - .setOutputType(testCase.datasetType) - val pipelineCmd = PipelineCommand - .newBuilder() - .setDefineOutput(defineDataset) - .build() - - val res = sendPlan(buildPlanFromPipelineCommand(pipelineCmd)).getPipelineCommandResult - assert(res !== PipelineCommandResult.getDefaultInstance) - assert(res.hasDefineOutputResult) - val graphResult = res.getDefineOutputResult - val identifier = graphResult.getResolvedIdentifier + // Build DefineOutput with the created graphId and dataset info + val defineDataset = DefineOutput + .newBuilder() + .setDataflowGraphId(graphId) + .setOutputName(testCase.datasetName) + .setOutputType(testCase.datasetType) + val pipelineCmd = PipelineCommand + .newBuilder() + .setDefineOutput(defineDataset) + .build() - assert(identifier.getCatalogName == testCase.expectedResolvedCatalog) - assert(identifier.getNamespaceList.asScala == testCase.expectedResolvedNamespace) - assert(identifier.getTableName == testCase.expectedResolvedDatasetName) + val res = sendPlan(buildPlanFromPipelineCommand(pipelineCmd)).getPipelineCommandResult + assert(res !== PipelineCommandResult.getDefaultInstance) + assert(res.hasDefineOutputResult) + val graphResult = res.getDefineOutputResult + val identifier = graphResult.getResolvedIdentifier + + assert(identifier.getCatalogName == testCase.expectedResolvedCatalog) + assert(identifier.getNamespaceList.asScala == testCase.expectedResolvedNamespace) + assert(identifier.getTableName == testCase.expectedResolvedDatasetName) + } finally { + sql(s"DROP DATABASE IF EXISTS spark_catalog.${testCase.defaultDatabase}") + } } } @@ -645,7 +675,6 @@ class SparkDeclarativePipelinesServerSuite name: String, datasetType: OutputType, flowName: String, - defaultCatalog: String, defaultDatabase: String, expectedResolvedFlowName: String, expectedResolvedCatalog: String, @@ -656,7 +685,6 @@ class SparkDeclarativePipelinesServerSuite name = "MV", datasetType = OutputType.MATERIALIZED_VIEW, flowName = "`mv`", - defaultCatalog = "`spark_catalog`", defaultDatabase = "`default`", expectedResolvedFlowName = "mv", expectedResolvedCatalog = "spark_catalog", @@ -665,7 +693,6 @@ class SparkDeclarativePipelinesServerSuite name = "TV", datasetType = OutputType.TEMPORARY_VIEW, flowName = "tv", - defaultCatalog = "spark_catalog", defaultDatabase = "default", expectedResolvedFlowName = "tv", expectedResolvedCatalog = "", @@ -676,16 +703,14 @@ class SparkDeclarativePipelinesServerSuite name = "MV custom", datasetType = OutputType.MATERIALIZED_VIEW, flowName = "mv", - defaultCatalog = "custom_catalog", defaultDatabase = "custom_db", expectedResolvedFlowName = "mv", - expectedResolvedCatalog = "custom_catalog", + expectedResolvedCatalog = "spark_catalog", expectedResolvedNamespace = Seq("custom_db")), DefineFlowTestCase( name = "TV custom", datasetType = OutputType.TEMPORARY_VIEW, flowName = "tv", - defaultCatalog = "custom_catalog", defaultDatabase = "custom_db", expectedResolvedFlowName = "tv", expectedResolvedCatalog = "", @@ -756,68 +781,73 @@ class SparkDeclarativePipelinesServerSuite namedGridTest("DefineFlow returns resolved data name for custom catalog/schema")( defineFlowCustomTests) { testCase => withRawBlockingStub { implicit stub => - val graphId = sendPlan( - buildCreateDataflowGraphPlan( - proto.PipelineCommand.CreateDataflowGraph + sql(s"CREATE DATABASE IF NOT EXISTS spark_catalog.${testCase.defaultDatabase}") + try { + val graphId = sendPlan( + buildCreateDataflowGraphPlan( + proto.PipelineCommand.CreateDataflowGraph + .newBuilder() + .setDefaultCatalog("spark_catalog") + .setDefaultDatabase(testCase.defaultDatabase) + .build())).getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId + assert(graphId.nonEmpty) + + // If the dataset type is TEMPORARY_VIEW, define the dataset explicitly first + if (testCase.datasetType == OutputType.TEMPORARY_VIEW) { + val defineDataset = DefineOutput .newBuilder() - .setDefaultCatalog(testCase.defaultCatalog) - .setDefaultDatabase(testCase.defaultDatabase) - .build())).getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId - assert(graphId.nonEmpty) + .setDataflowGraphId(graphId) + .setOutputName(testCase.flowName) + .setOutputType(OutputType.TEMPORARY_VIEW) - // If the dataset type is TEMPORARY_VIEW, define the dataset explicitly first - if (testCase.datasetType == OutputType.TEMPORARY_VIEW) { - val defineDataset = DefineOutput + val defineDatasetCmd = PipelineCommand + .newBuilder() + .setDefineOutput(defineDataset) + .build() + + val datasetRes = + sendPlan(buildPlanFromPipelineCommand(defineDatasetCmd)).getPipelineCommandResult + assert(datasetRes.hasDefineOutputResult) + } + + val defineFlow = DefineFlow .newBuilder() .setDataflowGraphId(graphId) - .setOutputName(testCase.flowName) - .setOutputType(OutputType.TEMPORARY_VIEW) - - val defineDatasetCmd = PipelineCommand + .setFlowName(testCase.flowName) + .setTargetDatasetName(testCase.flowName) + .setRelationFlowDetails( + DefineFlow.WriteRelationFlowDetails + .newBuilder() + .setRelation( + Relation + .newBuilder() + .setUnresolvedTableValuedFunction( + UnresolvedTableValuedFunction + .newBuilder() + .setFunctionName("range") + .addArguments(Expression + .newBuilder() + .setLiteral(Expression.Literal.newBuilder().setInteger(5).build()) + .build()) + .build()) + .build()) + .build()) + .build() + val pipelineCmd = PipelineCommand .newBuilder() - .setDefineOutput(defineDataset) + .setDefineFlow(defineFlow) .build() - - val datasetRes = - sendPlan(buildPlanFromPipelineCommand(defineDatasetCmd)).getPipelineCommandResult - assert(datasetRes.hasDefineOutputResult) + val res = sendPlan(buildPlanFromPipelineCommand(pipelineCmd)).getPipelineCommandResult + assert(res.hasDefineFlowResult) + val graphResult = res.getDefineFlowResult + val identifier = graphResult.getResolvedIdentifier + + assert(identifier.getCatalogName == testCase.expectedResolvedCatalog) + assert(identifier.getNamespaceList.asScala == testCase.expectedResolvedNamespace) + assert(identifier.getTableName == testCase.expectedResolvedFlowName) + } finally { + sql(s"DROP DATABASE IF EXISTS spark_catalog.${testCase.defaultDatabase}") } - - val defineFlow = DefineFlow - .newBuilder() - .setDataflowGraphId(graphId) - .setFlowName(testCase.flowName) - .setTargetDatasetName(testCase.flowName) - .setRelationFlowDetails( - DefineFlow.WriteRelationFlowDetails - .newBuilder() - .setRelation( - Relation - .newBuilder() - .setUnresolvedTableValuedFunction( - UnresolvedTableValuedFunction - .newBuilder() - .setFunctionName("range") - .addArguments(Expression - .newBuilder() - .setLiteral(Expression.Literal.newBuilder().setInteger(5).build()) - .build()) - .build()) - .build()) - .build()) - .build() - val pipelineCmd = PipelineCommand - .newBuilder() - .setDefineFlow(defineFlow) - .build() - val res = sendPlan(buildPlanFromPipelineCommand(pipelineCmd)).getPipelineCommandResult - assert(res.hasDefineFlowResult) - val graphResult = res.getDefineFlowResult - val identifier = graphResult.getResolvedIdentifier - - assert(identifier.getCatalogName == testCase.expectedResolvedCatalog) - assert(identifier.getNamespaceList.asScala == testCase.expectedResolvedNamespace) - assert(identifier.getTableName == testCase.expectedResolvedFlowName) } } } From 5a25b050e2f248637f4a7f8c88c6ad0dd0628df7 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 14 Nov 2025 16:46:22 -0800 Subject: [PATCH 146/400] [SPARK-54358][SDP] Checkpoint dirs collide when streaming tables in different schemas have same name ### What changes were proposed in this pull request? Updates the per-streaming table checkpoint path to use the fully qualified table path, instead of just its name. ### Why are the changes needed? A streaming table is a table fed by a stream. Streaming tables have checkpoint directories underneath their pipeline's storage root. These directories don't currently take the table namespace into account, which means that two tables with different schemas but the same name will be mapped to the same checkpoint directory. This could be very bad and cause data loss. ### Does this PR introduce _any_ user-facing change? Yes, but for an unreleased feature. ### How was this patch tested? Added a test for the collision case. Verified that it fails with the prior logic and now passes. ### Was this patch authored or co-authored using generative AI tooling? Closes #53070 from sryza/collide. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit e09c9994910704a54aca76953a80b0824812e620) Signed-off-by: Dongjoon Hyun --- .../sql/pipelines/graph/SystemMetadata.scala | 6 ++-- .../pipelines/graph/SystemMetadataSuite.scala | 35 ++++++++++++++++++- 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala index a9db28c331241..e3c93f97a54d5 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala @@ -40,14 +40,14 @@ case class FlowSystemMetadata( * which is storage/_checkpoints/flow_destination_table/flow_name. * @return the checkpoint root directory for `flow` */ - private def flowCheckpointsDirOpt(): Option[Path] = { + def flowCheckpointsDirOpt(): Option[Path] = { Option(if (graph.table.contains(flow.destinationIdentifier) || graph.sink.contains(flow.destinationIdentifier)) { val checkpointRoot = new Path(context.storageRoot, "_checkpoints") - val flowTableName = flow.destinationIdentifier.table + val flowTableId = flow.destinationIdentifier.nameParts.mkString(Path.SEPARATOR) val flowName = flow.identifier.table val checkpointDir = new Path( - new Path(checkpointRoot, flowTableName), + new Path(checkpointRoot, flowTableId), flowName ) logInfo( diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala index 71a4b7f68404a..c37a6fb52f95d 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala @@ -227,6 +227,38 @@ class SystemMetadataSuite updateContext2 ) } + + test("checkpoint dirs for tables with same name but different schema don't collide") { + val session = spark + import session.implicits._ + + // create a pipeline with only a single ST + val graph = new TestGraphRegistrationContext(spark) { + implicit val sparkSession: SparkSession = spark + val mem: MemoryStream[Int] = MemoryStream[Int] + mem.addData(1, 2, 3) + registerView("a", query = dfFlowFunc(mem.toDF())) + registerTable("st") + registerFlow("st", "st", query = readStreamFlowFunc("a")) + registerTable("schema2.st") + registerFlow("schema2.st", "schema2.st", query = readStreamFlowFunc("a")) + }.toDataflowGraph + + val updateContext = TestPipelineUpdateContext( + unresolvedGraph = graph, + spark = spark, + storageRoot = storageRoot, + failOnErrorEvent = true + ) + + val stFlow = graph.flow(fullyQualifiedIdentifier("st")) + val schema2StFlow = graph.flow(fullyQualifiedIdentifier("st", database = Option("schema2"))) + val stSystemMetadata = FlowSystemMetadata(updateContext, stFlow, graph) + val schema2StSystemMetadata = FlowSystemMetadata(updateContext, schema2StFlow, graph) + assert( + stSystemMetadata.flowCheckpointsDirOpt() != schema2StSystemMetadata.flowCheckpointsDirOpt() + ) + } } trait SystemMetadataTestHelpers { @@ -242,8 +274,9 @@ trait SystemMetadataTestHelpers { ): Path = { val expectedRawCheckPointDir = tableOrSinkElement match { case t if t.isInstanceOf[Table] || t.isInstanceOf[Sink] => + val tableId = t.identifier.nameParts.mkString(Path.SEPARATOR) new Path(updateContext.storageRoot) - .suffix(s"/_checkpoints/${t.identifier.table}/${flowElement.identifier.table}") + .suffix(s"/_checkpoints/$tableId/${flowElement.identifier.table}") .toString case _ => fail( From c47f239db1a802ed46814cae215097ae102ae4ba Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 15 Nov 2025 15:07:18 -0800 Subject: [PATCH 147/400] [SPARK-54355][CONNECT] Make `spark.connect.session.planCompression.defaultAlgorithm` to support `NONE` ### What changes were proposed in this pull request? This PR aims to make `spark.connect.session.planCompression.defaultAlgorithm` to support `NONE` additionally. ### Why are the changes needed? **BEFORE** ``` $ bin/spark-connect-shell -c spark.connect.session.planCompression.defaultAlgorithm=NONE ... $ scala> spark.range(1).count() ... Caused by: org.apache.spark.SparkIllegalArgumentException: [INVALID_CONF_VALUE.OUT_OF_RANGE_OF_OPTIONS] The value 'NONE' in the config "spark.connect.session.planCompression.defaultAlgorithm" is invalid. It should be one of 'ZSTD'. SQLSTATE: 22022 ``` **AFTER** ``` $ bin/spark-connect-shell -c spark.connect.session.planCompression.defaultAlgorithm=NONE ... scala> spark.range(1).count() val res0: Long = 1 ``` ### Does this PR introduce _any_ user-facing change? No behavior change because this is a new option for a new feature. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53068 from dongjoon-hyun/SPARK-54355. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit e3e9863737357020391ead0dadcee9fd50b2e014) Signed-off-by: Dongjoon Hyun --- .../sql/connect/config/ConnectPlanCompressionAlgorithm.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala index 0f9b959ee7256..4052627fd8c3b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala @@ -17,5 +17,5 @@ package org.apache.spark.sql.connect.config object ConnectPlanCompressionAlgorithm extends Enumeration { - val ZSTD = Value + val ZSTD, NONE = Value } From 8fb5285810acb6329e57b6a4a81460484c0b807c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 15 Nov 2025 15:42:50 -0800 Subject: [PATCH 148/400] [SPARK-54366][INFRA] Add `free_disk_space` step to K8s integration test GitHub Action job ### What changes were proposed in this pull request? This PR aims to add `free_disk_space` step to K8s integration test GitHub Action job. ### Why are the changes needed? The K8s integration test CI is flaky due to `No space left on device` error. - https://github.com/apache/spark/actions/runs/19354883389/job/55448531341 ``` [info] 25/11/14 21:27:02 ERROR TaskSchedulerImpl: Lost executor 4 on 10.244.0.67: Unable to create executor due to /var/data/spark-163899dd-08da-4b76-b71d-c428207a3bdf/spark-1e9d976f-69b3-4274-af03-300cfc4d6fd5/-14621403551763155568738_cache -> ./software.amazon.awssdk_bundle-2.29.52.jar: No space left on device ``` Like the other four GitHub Action jobs, `free_disk_space_container` will mitigate this situation in this job. **BEFORE** ``` $ git grep 'free_disk_space$' .github/workflows/build_and_test.yml: ./dev/free_disk_space .github/workflows/release.yml: ./dev/free_disk_space ``` **AFTER** ``` $ git grep 'free_disk_space$' .github/workflows/build_and_test.yml: ./dev/free_disk_space .github/workflows/build_and_test.yml: ./dev/free_disk_space .github/workflows/release.yml: ./dev/free_disk_space ``` ### Does this PR introduce _any_ user-facing change? No, this is a test infra change. ### How was this patch tested? Pass the CIs and check the log. The following is the log result of this PR. - https://github.com/dongjoon-hyun/spark/actions/runs/19395758483/job/55495933312 **BEFORE CLEANUP** ``` + df -h Filesystem Size Used Avail Use% Mounted on /dev/root 72G 54G 18G 76% / tmpfs 7.9G 84K 7.9G 1% /dev/shm tmpfs 3.2G 1.1M 3.2G 1% /run tmpfs 5.0M 0 5.0M 0% /run/lock /dev/sda16 881M 62M 758M 8% /boot /dev/sda15 105M 6.2M 99M 6% /boot/efi /dev/sdb1 74G 4.1G 66G 6% /mnt tmpfs 1.6G 12K 1.6G 1% /run/user/1001 ``` **AFTER CLEANUP** ``` + df -h Filesystem Size Used Avail Use% Mounted on /dev/root 72G 21G 52G 29% / tmpfs 7.9G 84K 7.9G 1% /dev/shm tmpfs 3.2G 1.1M 3.2G 1% /run tmpfs 5.0M 0 5.0M 0% /run/lock /dev/sda16 881M 62M 758M 8% /boot /dev/sda15 105M 6.2M 99M 6% /boot/efi /dev/sdb1 74G 4.1G 66G 6% /mnt tmpfs 1.6G 12K 1.6G 1% /run/user/1001 ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53077 from dongjoon-hyun/SPARK-54366. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 0311f44e33e5cf8ba60ccc330de3df4f688f5847) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 161164dc5866b..f0d2acbacff7f 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1314,6 +1314,11 @@ jobs: key: k8s-integration-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} restore-keys: | k8s-integration-coursier- + - name: Free up disk space + run: | + if [ -f ./dev/free_disk_space ]; then + ./dev/free_disk_space + fi - name: Install Java ${{ inputs.java }} uses: actions/setup-java@v4 with: From 827df6ed694f596d4d7bc2a46f44fdfcdb5c47c0 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sat, 15 Nov 2025 21:58:40 -0800 Subject: [PATCH 149/400] [SPARK-53924] Reload DSv2 tables in views created using plans on each view access MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR makes Spark reload DSv2 tables in views created using plans on each view access. ### Why are the changes needed? The current problem is that the view definition in the session catalog captures the analyzed plan that references `Table` (that is supposed to pin the version). If a connector doesn’t have an internal cache and produces a new `Table` object on each load, the table referenced in the view will become orphan and there will be no way to refresh it unless that `Table` instance auto refreshes on each scan (super dangerous). ### Does this PR introduce _any_ user-facing change? Yes, but it restores the correct behavior without requiring hacks in connectors. ### How was this patch tested? This PR comes with tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52876 from aokolnychyi/spark-53924. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun (cherry picked from commit 407e79c0ac52cb39561b9cb7af742e09b9f6c645) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 9 + .../sql/catalyst/analysis/Analyzer.scala | 24 +- .../analysis/RelationResolution.scala | 39 +++ .../catalyst/analysis/V2TableReference.scala | 139 ++++++++++ .../sql/connector/catalog/V2TableUtil.scala | 2 +- .../sql/errors/QueryCompilationErrors.scala | 26 ++ .../spark/sql/execution/CacheManager.scala | 4 + .../spark/sql/execution/command/views.scala | 14 +- .../apache/spark/sql/CachedTableSuite.scala | 73 ++++++ .../DataSourceV2DataFrameSuite.scala | 240 ++++++++++++++++++ 10 files changed, 564 insertions(+), 6 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 98c61aa33f2e2..308f49b097fc4 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2123,6 +2123,15 @@ ], "sqlState" : "42000" }, + "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION" : { + "message" : [ + "View plan references table whose columns changed since the view plan was initially captured.", + "Column changes:", + "", + "This indicates the table has evolved and the view based on the plan must be recreated." + ], + "sqlState" : "51024" + }, "INCOMPATIBLE_COLUMN_TYPE" : { "message" : [ " can only be performed on tables with compatible column types. The column of the table is type which is not compatible with at the same column of the first table.." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f9e14cb0daf8f..6b0665c1b7f35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1185,6 +1185,18 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } + private def resolveAsV2Relation(plan: LogicalPlan): Option[DataSourceV2Relation] = { + plan match { + case ref: V2TableReference => + EliminateSubqueryAliases(relationResolution.resolveReference(ref)) match { + case r: DataSourceV2Relation => Some(r) + case _ => None + } + case r: DataSourceV2Relation => Some(r) + case _ => None + } + } + def apply(plan: LogicalPlan) : LogicalPlan = plan.resolveOperatorsUpWithPruning(AlwaysProcess.fn, ruleId) { case i @ InsertIntoStatement(table, _, _, _, _, _, _) => @@ -1210,13 +1222,14 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor resolveRelation(u).map(unwrapRelationPlan).map { case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError( v.desc.identifier, write) - case r: DataSourceV2Relation => write.withNewTable(r) case u: UnresolvedCatalogRelation => throw QueryCompilationErrors.writeIntoV1TableNotAllowedError( u.tableMeta.identifier, write) - case other => - throw QueryCompilationErrors.writeIntoTempViewNotAllowedError( - u.multipartIdentifier.quoted) + case plan => + resolveAsV2Relation(plan).map(write.withNewTable).getOrElse { + throw QueryCompilationErrors.writeIntoTempViewNotAllowedError( + u.multipartIdentifier.quoted) + } }.getOrElse(write) case _ => write } @@ -1224,6 +1237,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case u: UnresolvedRelation => resolveRelation(u).map(resolveViews(_, u.options)).getOrElse(u) + case r: V2TableReference => + relationResolution.resolveReference(r) + case r @ RelationTimeTravel(u: UnresolvedRelation, timestamp, version) if timestamp.forall(ts => ts.resolved && !SubqueryExpression.hasSubquery(ts)) => val timeTravelSpec = TimeTravelSpec.create(timestamp, version, conf.sessionLocalTimeZone) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala index 6ff40da88ed18..c7b92bc2a9fe5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala @@ -225,6 +225,45 @@ class RelationResolution(override val catalogManager: CatalogManager) } } + def resolveReference(ref: V2TableReference): LogicalPlan = { + val relation = getOrLoadRelation(ref) + val planId = ref.getTagValue(LogicalPlan.PLAN_ID_TAG) + cloneWithPlanId(relation, planId) + } + + private def getOrLoadRelation(ref: V2TableReference): LogicalPlan = { + val key = toCacheKey(ref.catalog, ref.identifier) + relationCache.get(key) match { + case Some(cached) => + adaptCachedRelation(cached, ref) + case None => + val relation = loadRelation(ref) + relationCache.update(key, relation) + relation + } + } + + private def loadRelation(ref: V2TableReference): LogicalPlan = { + val table = ref.catalog.loadTable(ref.identifier) + V2TableReferenceUtils.validateLoadedTable(table, ref) + val tableName = ref.identifier.toQualifiedNameParts(ref.catalog) + SubqueryAlias(tableName, ref.toRelation(table)) + } + + private def adaptCachedRelation(cached: LogicalPlan, ref: V2TableReference): LogicalPlan = { + cached transform { + case r: DataSourceV2Relation if matchesReference(r, ref) => + V2TableReferenceUtils.validateLoadedTable(r.table, ref) + r.copy(output = ref.output, options = ref.options) + } + } + + private def matchesReference( + relation: DataSourceV2Relation, + ref: V2TableReference): Boolean = { + relation.catalog.contains(ref.catalog) && relation.identifier.contains(ref.identifier) + } + private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty private def isReferredTempViewName(nameParts: Seq[String]): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala new file mode 100644 index 0000000000000..b6a2c6db66044 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.V2TableReference.Context +import org.apache.spark.sql.catalyst.analysis.V2TableReference.TableInfo +import org.apache.spark.sql.catalyst.analysis.V2TableReference.TemporaryViewContext +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper +import org.apache.spark.sql.connector.catalog.Column +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.MetadataColumn +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.catalog.V2TableUtil +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits._ + +/** + * A reference to a V2 table. + * + * References are placeholders for the latest table metadata and are replaced with actual table + * versions during analysis, allowing Spark to reload tables with up-to-date metadata. The newly + * loaded table metadata is validated against the original metadata depending on the context. + * For instance, temporary views with fully resolved logical plans don't allow schema changes + * in underlying tables. + */ +private[sql] case class V2TableReference private( + catalog: TableCatalog, + identifier: Identifier, + options: CaseInsensitiveStringMap, + info: TableInfo, + output: Seq[AttributeReference], + context: Context) + extends LeafNode with MultiInstanceRelation with NamedRelation { + + override def name: String = V2TableUtil.toQualifiedName(catalog, identifier) + + override def newInstance(): V2TableReference = { + copy(output = output.map(_.newInstance())) + } + + override def computeStats(): Statistics = Statistics.DUMMY + + override def simpleString(maxFields: Int): String = { + val outputString = truncatedString(output, "[", ", ", "]", maxFields) + s"TableReference$outputString $name" + } + + def toRelation(table: Table): DataSourceV2Relation = { + DataSourceV2Relation(table, output, Some(catalog), Some(identifier), options) + } +} + +private[sql] object V2TableReference { + + case class TableInfo( + columns: Seq[Column], + metadataColumns: Seq[MetadataColumn]) + + sealed trait Context + case class TemporaryViewContext(viewName: Seq[String]) extends Context + + def createForTempView(relation: DataSourceV2Relation, viewName: Seq[String]): V2TableReference = { + create(relation, TemporaryViewContext(viewName)) + } + + private def create(relation: DataSourceV2Relation, context: Context): V2TableReference = { + val ref = V2TableReference( + relation.catalog.get.asTableCatalog, + relation.identifier.get, + relation.options, + TableInfo( + columns = relation.table.columns.toImmutableArraySeq, + metadataColumns = V2TableUtil.extractMetadataColumns(relation)), + relation.output, + context) + ref.copyTagsFrom(relation) + ref + } +} + +private[sql] object V2TableReferenceUtils extends SQLConfHelper { + + def validateLoadedTable(table: Table, ref: V2TableReference): Unit = { + ref.context match { + case ctx: TemporaryViewContext => + validateLoadedTableInTempView(table, ref, ctx) + case ctx => + throw SparkException.internalError(s"Unknown table ref context: ${ctx.getClass.getName}") + } + } + + private def validateLoadedTableInTempView( + table: Table, + ref: V2TableReference, + ctx: TemporaryViewContext): Unit = { + val tableName = ref.identifier.toQualifiedNameParts(ref.catalog) + + val dataErrors = V2TableUtil.validateCapturedColumns(table, ref.info.columns) + if (dataErrors.nonEmpty) { + throw QueryCompilationErrors.columnsChangedAfterViewWithPlanCreation( + ctx.viewName, + tableName, + dataErrors) + } + + val metaErrors = V2TableUtil.validateCapturedMetadataColumns(table, ref.info.metadataColumns) + if (metaErrors.nonEmpty) { + throw QueryCompilationErrors.metadataColumnsChangedAfterViewWithPlanCreation( + ctx.viewName, + tableName, + metaErrors) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala index 9873fc0881a05..fa8ed1b21a313 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala @@ -97,7 +97,7 @@ private[sql] object V2TableUtil extends SQLConfHelper { } // extracts original column info for all metadata attributes in relation - private def extractMetadataColumns(relation: DataSourceV2Relation): Seq[MetadataColumn] = { + def extractMetadataColumns(relation: DataSourceV2Relation): Seq[MetadataColumn] = { val metaAttrs = relation.output.filter(_.isMetadataCol) if (metaAttrs.nonEmpty) { val metaCols = metadataColumns(relation.table) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 29b87e2d00968..f741c5c3975e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4466,4 +4466,30 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat ) ) } + + def columnsChangedAfterViewWithPlanCreation( + viewName: Seq[String], + tableName: Seq[String], + errors: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + messageParameters = Map( + "viewName" -> toSQLId(viewName), + "tableName" -> toSQLId(tableName), + "colType" -> "data", + "errors" -> errors.mkString("\n- ", "\n- ", ""))) + } + + def metadataColumnsChangedAfterViewWithPlanCreation( + viewName: Seq[String], + tableName: Seq[String], + errors: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + messageParameters = Map( + "viewName" -> toSQLId(viewName), + "tableName" -> toSQLId(tableName), + "colType" -> "metadata", + "errors" -> errors.mkString("\n- ", "\n- ", ""))) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 671fcb765648d..3944cf818895e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.analysis.V2TableReference import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint @@ -250,6 +251,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val nameInCache = v2Ident.toQualifiedNameParts(catalog) isSameName(nameInCache) && (includeTimeTravel || timeTravelSpec.isEmpty) + case r: V2TableReference => + isSameName(r.identifier.toQualifiedNameParts(r.catalog)) + case v: View => isSameName(v.desc.identifier.nameParts) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 514b64f6abed2..11ec17ca57fd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CapturesConfig, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, GlobalTempView, LocalTempView, SchemaEvolution, SchemaUnsupported, ViewSchemaMode, ViewType} +import org.apache.spark.sql.catalyst.analysis.V2TableReference import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, TemporaryViewRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{AnalysisOnlyCommand, CreateTempView, CTEInChildren, CTERelationDef, LogicalPlan, Project, View, WithCTE} @@ -34,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.types.{MetadataBuilder, StructType} import org.apache.spark.sql.util.SchemaUtils @@ -733,7 +735,17 @@ object ViewHelper extends SQLConfHelper with Logging with CapturesConfig { } else { TemporaryViewRelation( prepareTemporaryViewStoringAnalyzedPlan(name, aliasedPlan, defaultCollation), - Some(aliasedPlan)) + Some(prepareTemporaryViewPlan(name, aliasedPlan))) + } + } + + private def prepareTemporaryViewPlan( + viewName: TableIdentifier, + plan: LogicalPlan): LogicalPlan = { + plan transform { + case r: DataSourceV2Relation + if r.catalog.isDefined && r.identifier.isDefined && r.timeTravelSpec.isEmpty => + V2TableReference.createForTempView(r, viewName.nameParts) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 191587888ab81..91addd72ab2b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -2457,6 +2457,79 @@ class CachedTableSuite extends QueryTest with SQLTestUtils } } + test("SPARK-53924: insert into DSv2 table invalidates cache of SQL temp view (plan)") { + checkInsertInvalidatesCacheOfSQLTempView(storePlan = true) + } + + test("SPARK-53924: insert into DSv2 table invalidates cache of SQL temp view (text)") { + checkInsertInvalidatesCacheOfSQLTempView(storePlan = false) + } + + private def checkInsertInvalidatesCacheOfSQLTempView(storePlan: Boolean): Unit = { + val t = "testcat.tbl" + withTable(t, "v") { + withSQLConf(SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW.key -> storePlan.toString) { + sql(s"CREATE TABLE $t (id int, data string) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a'), (2, 'b')") + + // create and cache SQL temp view + sql(s"CREATE TEMPORARY VIEW v AS SELECT id FROM $t") + sql("SELECT * FROM v").cache() + + // verify view is cached + assertCached(sql("SELECT * FROM v")) + checkAnswer(sql("SELECT * FROM v"), Seq(Row(1), Row(2))) + + // insert data into base table + sql(s"INSERT INTO $t VALUES (3, 'c'), (4, 'd')") + + // verify cache was refreshed and will pick up new data + checkCacheLoading(sql(s"SELECT * FROM v"), isLoaded = false) + + // verify view is recached correctly + assertCached(sql("SELECT * FROM v")) + checkAnswer( + sql("SELECT * FROM v"), + Seq(Row(1), Row(2), Row(3), Row(4))) + } + } + } + + test("SPARK-53924: uncache DSv2 table uncaches SQL temp views (plan)") { + checkUncacheTableUncachesSQLTempView(storePlan = true) + } + + test("SPARK-53924: uncache DSv2 table uncaches SQL temp views (text)") { + checkUncacheTableUncachesSQLTempView(storePlan = false) + } + + private def checkUncacheTableUncachesSQLTempView(storePlan: Boolean): Unit = { + val t = "testcat.tbl" + withTable(t, "v") { + withSQLConf(SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW.key -> storePlan.toString) { + sql(s"CREATE TABLE $t (id int, data string) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a'), (2, 'b')") + + // cache table + sql(s"CACHE TABLE $t") + assertCached(sql(s"SELECT * FROM $t")) + checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(1, "a"), Row(2, "b"))) + + // create and cache SQL temp view + sql(s"CREATE TEMPORARY VIEW v AS SELECT id FROM $t") + sql("SELECT * FROM v").cache() + assertCached(sql("SELECT * FROM v")) + checkAnswer(sql("SELECT * FROM v"), Seq(Row(1), Row(2))) + + // uncache table must invalidate view cache (cascading) + sql(s"UNCACHE TABLE $t") + + // verify view is not cached anymore + assertNotCached(sql("SELECT * FROM v")) + } + } + } + test("uncache persistent table via catalog API") { withTable("tbl1") { sql("CREATE TABLE tbl1 (name STRING, age INT) USING parquet") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 94fc250f4f831..c59e624cb1784 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1352,6 +1352,246 @@ class DataSourceV2DataFrameSuite } } + test("SPARK-53924: temp view on DSv2 table detects added columns") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + // create temp view using DataFrame API + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // add column to underlying table + sql(s"ALTER TABLE $t ADD COLUMN age int") + + // accessing temp view should detect schema change + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> "\n- `age` INT has been added")) + } + } + + test("SPARK-53924: temp view on DSv2 table detects removed columns") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, age int) USING foo") + + // create temp view + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // drop column from underlying table + sql(s"ALTER TABLE $t DROP COLUMN age") + + // accessing temp view should detect schema change + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> "\n- `age` INT has been removed")) + } + } + + test("SPARK-53924: temp view on DSv2 table detects nullability changes") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string NOT NULL) USING foo") + + // create temp view + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // change nullability constraint using ALTER TABLE + sql(s"ALTER TABLE $t ALTER COLUMN data DROP NOT NULL") + + // accessing temp view should detect schema change + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> "\n- `data` type has changed from STRING NOT NULL to STRING")) + } + } + + test("SPARK-53924: temp view on DSv2 table accepts table ID changes") { + val t = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + val df = Seq((1L, "a"), (2L, "b")).toDF("id", "data") + df.write.insertInto(t) + + // create temp view + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), df) + + // capture the original table ID + val originalTableId = catalog("testcat").loadTable(ident).id + + // drop and recreate table (this changes the table ID) + sql(s"DROP TABLE $t") + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + // verify table ID changed + val newTableId = catalog("testcat").loadTable(ident).id + assert(originalTableId != newTableId) + + // accessing temp view should work despite table ID change (returns empty data) + checkAnswer(spark.table("v"), Seq.empty) + + // insert new data and verify view reflects it + val newDF = Seq((3L, "c"), (4L, "d")).toDF("id", "data") + newDF.write.insertInto(t) + checkAnswer(spark.table("v"), newDF) + } + } + + test("SPARK-53924: createOrReplaceTempView works after schema change") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint) USING foo") + + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // alter table + sql(s"ALTER TABLE $t ADD COLUMN data string") + + // old view fails + intercept[AnalysisException] { spark.table("v").collect() } + + // recreate view with updated schema + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // now it should work with new schema + val df = Seq((1L, "a"), (2L, "b")).toDF("id", "data") + df.write.insertInto(t) + checkAnswer(spark.table("v"), df) + } + } + + + test("SPARK-53924: temp view on DSv2 table with read options") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + // create temp view with options + val df = spark.read.option("fakeOption", "testValue").table(t) + df.createOrReplaceTempView("v") + + // verify options are preserved in the view + val options = spark.table("v").queryExecution.analyzed.collectFirst { + case d: DataSourceV2Relation => d.options + }.get + assert(options.get("fakeOption") == "testValue") + + // schema changes should still be detected + sql(s"ALTER TABLE $t ADD COLUMN age int") + + // accessing temp view should detect schema change + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> "\n- `age` INT has been added")) + } + } + + test("SPARK-53924: temp view on DSv2 table created using SQL with plan detects changes") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + withSQLConf(SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW.key -> "true") { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + // create temp view using SQL that should capture plan + sql(s"CREATE OR REPLACE TEMPORARY VIEW v AS SELECT * FROM $t") + checkAnswer(spark.table("v"), Seq.empty) + + // verify that view stores analyzed plan + val Some(view) = spark.sessionState.catalog.getRawTempView("v") + assert(view.plan.isDefined) + + // add column to underlying table + sql(s"ALTER TABLE $t ADD COLUMN age int") + + // accessing temp view should detect schema change + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> "\n- `age` INT has been added")) + } + } + } + + test("SPARK-53924: temp view on DSv2 table detects VARCHAR/CHAR type changes") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, name VARCHAR(10)) USING foo") + + // create temp view + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // change VARCHAR(10) to VARCHAR(20) + sql(s"ALTER TABLE $t ALTER COLUMN name TYPE VARCHAR(20)") + + // accessing temp view should detect type change + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> "\n- `name` type has changed from VARCHAR(10) to VARCHAR(20)")) + } + } + + test("SPARK-53924: temp view on DSv2 table works after inserting data") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + // create temp view + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // insert data into underlying table (no schema change) + val df = Seq((1L, "a"), (2L, "b")).toDF("id", "data") + df.write.insertInto(t) + + // accessing temp view should work and reflect new data + checkAnswer(spark.table("v"), df) + + // insert more data + val df2 = Seq((3L, "c"), (4L, "d")).toDF("id", "data") + df2.write.insertInto(t) + + // view should reflect all data + checkAnswer(spark.table("v"), df.union(df2)) + } + } + private def pinTable(catalogName: String, ident: Identifier, version: String): Unit = { catalog(catalogName) match { case inMemory: BasicInMemoryTableCatalog => inMemory.pinTable(ident, version) From c1cd4316884cbb2bb45fe030512c13cd733c19c9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 15 Nov 2025 22:03:22 -0800 Subject: [PATCH 150/400] [SPARK-54371][INFRA] Fix `spark-rm` Dockefile to install `pkgdown` version at the end ### What changes were proposed in this pull request? This PR aims to fix `spark-rm` Dockefile to install `pkgdown` version at the end. ### Why are the changes needed? Although `pkgdown` is supposed to be `2.0.1`, it's changed at the next package installation like the following. We should install `pkgdown` at the end to make it sure. https://github.com/apache/spark/blob/0311f44e33e5cf8ba60ccc330de3df4f688f5847/dev/create-release/spark-rm/Dockerfile#L89 - https://github.com/apache/spark/actions/workflows/release.yml - https://github.com/apache/spark/actions/runs/19386198324/job/55473421715 ``` #11 1007.3 Downloading package from url: https://cloud.r-project.org/src/contrib/Archive/preferably/preferably_0.4.tar.gz #11 1008.9 pkgdown (2.0.1 -> 2.2.0) [CRAN] #11 1008.9 Installing 1 packages: pkgdown #11 1008.9 Installing package into '/usr/local/lib/R/site-library' #11 1008.9 (as 'lib' is unspecified) #11 1009.4 trying URL 'https://cloud.r-project.org/src/contrib/pkgdown_2.2.0.tar.gz' #11 1009.7 Content type 'application/x-gzip' length 1280630 bytes (1.2 MB) #11 1009.7 ================================================== #11 1009.7 downloaded 1.2 MB #11 1009.7 #11 1010.2 * installing *source* package 'pkgdown' ... #11 1010.2 ** package 'pkgdown' successfully unpacked and MD5 sums checked #11 1010.2 ** using staged installation #11 1010.3 ** R #11 1010.3 ** inst #11 1010.3 ** byte-compile and prepare package for lazy loading #11 1013.1 ** help #11 1013.2 *** installing help indices #11 1013.2 *** copying figures #11 1013.2 ** building package indices #11 1013.5 ** installing vignettes #11 1013.5 ** testing if installed package can be loaded from temporary location #11 1013.8 ** testing if installed package can be loaded from final location #11 1014.1 ** testing if installed package keeps a record of temporary installation path #11 1014.1 * DONE (pkgdown) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ``` $ dev/create-release/do-release-docker.sh -d /tmp/spark-4.1.0 -n -s docs $ docker run -it --rm --entrypoint /bin/bash spark-rm spark-rm923a388425fa:/opt/spark-rm/output$ Rscript -e 'installed.packages()' | grep pkgdown | head -n1 pkgdown "pkgdown" "/usr/local/lib/R/site-library" "2.0.1" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53083 from dongjoon-hyun/SPARK-54371. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 2e4708ed10b755f9229ac570b2b22534bd099aea) Signed-off-by: Dongjoon Hyun --- dev/create-release/spark-rm/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index af8c34852ca03..e7a11d48c33f1 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -86,8 +86,8 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ 'ggplot2', 'mvtnorm', 'statmod', 'xml2'), repos='https://cloud.r-project.org/')" && \ Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='https://cloud.r-project.org')" && \ Rscript -e "devtools::install_version('lintr', version='2.0.1', repos='https://cloud.r-project.org')" && \ - Rscript -e "devtools::install_version('pkgdown', version='2.0.1', repos='https://cloud.r-project.org')" && \ - Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" + Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('pkgdown', version='2.0.1', repos='https://cloud.r-project.org')" # See more in SPARK-39735 ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" From e70177a2054e88fbec47e0d3ea3d1e8a6ba4fcd7 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 16 Nov 2025 07:56:03 -0800 Subject: [PATCH 151/400] [SPARK-54369][CONNECT][TESTS] Fix `PythonPipelineSuite` flakiness via `Set` instead of `Seq` ### What changes were proposed in this pull request? This PR aims to fix `PythonPipelineSuite` flakiness via `Set` instead of `Seq` in multiple places. ### Why are the changes needed? Currently, `PythonPipelineSuite` is flaky like the following. We should fix this flakiness. - https://github.com/apache/spark/actions/runs/19396864076/job/55498096472 ``` [info] - referencing internal datasets *** FAILED *** (821 milliseconds) [info] List(`spark_catalog`.`default`.`src`, `spark_catalog`.`default`.`c`, `spark_catalog`.`default`.`a`) did not equal List(`spark_catalog`.`default`.`src`, `spark_catalog`.`default`.`a`, `spark_catalog`.`default`.`c`) (PythonPipelineSuite.scala:366) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53080 from dongjoon-hyun/SPARK-XXX. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 13545a4e507827c519faebaf0c64ad774eec22d3) Signed-off-by: Dongjoon Hyun --- .../sql/connect/pipelines/PythonPipelineSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala index 1850241f07026..45d8c7b18b846 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala @@ -364,11 +364,13 @@ class PythonPipelineSuite val (streamingFlows, batchFlows) = graph.resolvedFlows.partition(_.df.isStreaming) assert( - batchFlows.map(_.identifier) == Seq( + batchFlows.map(_.identifier).toSet == Set( graphIdentifier("src"), graphIdentifier("a"), graphIdentifier("c"))) - assert(streamingFlows.map(_.identifier) == Seq(graphIdentifier("b"), graphIdentifier("d"))) + assert( + streamingFlows.map(_.identifier).toSet == + Set(graphIdentifier("b"), graphIdentifier("d"))) } test("referencing external datasets") { @@ -722,7 +724,8 @@ class PythonPipelineSuite assert( graph .flowsTo(graphIdentifier("a")) - .map(_.identifier) == Seq(graphIdentifier("a"), graphIdentifier("something"))) + .map(_.identifier) + .toSet == Set(graphIdentifier("a"), graphIdentifier("something"))) } test("groupby and rollup works with internal datasets, referencing with (col, str)") { From a54411d6804f0cbaf2b3b08a0c0f54e4b6c59d96 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 16 Nov 2025 07:57:27 -0800 Subject: [PATCH 152/400] [SPARK-54370][INFRA] Limit the Maven GitHub Action job timeout to 150 minutes ### What changes were proposed in this pull request? This PR aims to limit the Maven GitHub Action job timeout to 150 minutes. ### Why are the changes needed? Currently, Maven CI runs 6 hours which is the default timeout. In general, this job should pass in 150 minutes. Screenshot 2025-11-15 at 18 44 28 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53081 from dongjoon-hyun/SPARK-54370. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 0a42f557a8d2e952d7a651faa0843629f3ceffff) Signed-off-by: Dongjoon Hyun --- .github/workflows/maven_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 666a504ad0732..74bd818a522dc 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -56,6 +56,7 @@ jobs: build: name: "Build modules using Maven: ${{ matrix.modules }} ${{ matrix.comment }}" runs-on: ${{ inputs.os }} + timeout-minutes: 150 strategy: fail-fast: false matrix: From 7cd3f858c8aa3d5cf9be96aafabd851652844bfe Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Mon, 17 Nov 2025 11:39:11 +0800 Subject: [PATCH 153/400] [SPARK-54319][SQL] BHJ LeftAnti update numOutputRows wrong when codegen is disabled ### What changes were proposed in this pull request? BHJ LeftAnti update numOutputRows missing case for hashed = EmptyHashedRelation image ### Why are the changes needed? Fix missing sql metrics for BHJ ### Does this PR introduce _any_ user-facing change? Yes, BHJ LeftAnti will update numOutputRows when hashed = EmptyHashedRelation ### How was this patch tested? Existed UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #53014 from AngersZhuuuu/SPARK-54319. Authored-by: Angerszhuuuu Signed-off-by: Wenchen Fan (cherry picked from commit 3757091e1c513eb2390dc2b939bcf9e1fb70ef53) Signed-off-by: Wenchen Fan --- .../joins/BroadcastHashJoinExec.scala | 5 +++- .../execution/metric/SQLMetricsSuite.scala | 23 ++++++++++++++----- 2 files changed, 21 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 368534d05b1f0..b62d8f0798b6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -129,7 +129,10 @@ case class BroadcastHashJoinExec( val hashed = broadcastRelation.value.asReadOnlyCopy() TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) if (hashed == EmptyHashedRelation) { - streamedIter + streamedIter.map { row => + numOutputRows += 1 + row + } } else if (hashed == HashedRelationWithAllNullKeys) { Iterator.empty } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 4e50457ae47da..402365a59eceb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -915,16 +915,27 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils withTable("t1", "t2") { spark.range(4).write.saveAsTable("t1") spark.range(2).write.saveAsTable("t2") - val df = sql("SELECT * FROM t1 WHERE id NOT IN (SELECT id FROM t2)") - df.collect() - val plan = df.queryExecution.executedPlan + val df1 = sql("SELECT * FROM t1 WHERE id NOT IN (SELECT id FROM t2)") + df1.collect() + val plan1 = df1.queryExecution.executedPlan - val joins = plan.collect { + val joins1 = plan1.collect { case s: BroadcastHashJoinExec => s } - assert(joins.size === 1) - testMetricsInSparkPlanOperator(joins.head, Map("numOutputRows" -> 2)) + assert(joins1.size === 1) + testMetricsInSparkPlanOperator(joins1.head, Map("numOutputRows" -> 2)) + + val df2 = sql("SELECT * FROM t1 WHERE id NOT IN (SELECT id FROM t2 WHERE 1 = 2)") + df2.collect() + val plan2 = df2.queryExecution.executedPlan + + val joins2 = plan2.collect { + case s: BroadcastHashJoinExec => s + } + + assert(joins2.size === 1) + testMetricsInSparkPlanOperator(joins2.head, Map("numOutputRows" -> 4)) } } } From aed0547af11d3e1ca69c1f33f6e69acfb2308780 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 17 Nov 2025 16:40:42 +0800 Subject: [PATCH 154/400] [SPARK-54375][CONNECT][TESTS] Add `assume` to cases in `PythonPipelineSuite` to skip tests when PyConnect dependencies is not available ### What changes were proposed in this pull request? SPARK-54020 added some new test cases in `PythonPipelineSuite`. This pr incorporates `assume(PythonTestDepsChecker.isConnectDepsAvailable)` for these test cases to ensure that the tests are skipped rather than failing when PyConnect dependencies are missing. ### Why are the changes needed? Enhance the robustness of test cases. Prior to this, when executing `build/sbt "connect/testOnly org.apache.spark.sql.connect.pipelines.PythonPipelineSuite"`: ``` [info] - reading internal datasets outside query function that trigger eager analysis or execution will fail (spark.sql("SELECT * FROM src")) *** FAILED *** (4 milliseconds) [info] "org.apache.spark.sql.connect.PythonTestDepsChecker.isConnectDepsAvailable was false" did not contain "TABLE_OR_VIEW_NOT_FOUND" (PythonPipelineSuite.scala:546) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472) [info] at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471) [info] at org.scalatest.Assertions$.newAssertionFailedException(Assertions.scala:1231) [info] at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:1295) [info] at org.apache.spark.sql.connect.pipelines.PythonPipelineSuite.$anonfun$new$43(PythonPipelineSuite.scala:546) [info] at org.apache.spark.sql.connect.pipelines.PythonPipelineSuite.$anonfun$new$43$adapted(PythonPipelineSuite.scala:532) [info] at org.apache.spark.SparkFunSuite.$anonfun$gridTest$2(SparkFunSuite.scala:241) [info] at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18) [info] at org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127) [info] at org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282) [info] at org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231) [info] at org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230) ... [info] *** 24 TESTS FAILED *** [error] Failed tests: [error] org.apache.spark.sql.connect.pipelines.PythonPipelineSuite [error] (connect / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass Github Actions - Manually verify that the relevant tests will no longer fail when PyConnect dependencies are missing. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53088 from LuciferYang/SPARK-54375. Authored-by: yangjie01 Signed-off-by: yangjie01 (cherry picked from commit 722bcc0f0d15245a39fae62c0c1c764e4b6a02f8) Signed-off-by: yangjie01 --- .../spark/sql/connect/pipelines/PythonPipelineSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala index 45d8c7b18b846..826e2338589da 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala @@ -530,6 +530,7 @@ class PythonPipelineSuite "eager analysis or execution will fail")( Seq("""spark.sql("SELECT * FROM src")""", """spark.read.table("src").collect()""")) { command => + assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |@dp.materialized_view @@ -1047,6 +1048,7 @@ class PythonPipelineSuite gridTest("Unsupported SQL command outside query function should result in a failure")( unsupportedSqlCommandList) { unsupportedSqlCommand => + assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |spark.sql("$unsupportedSqlCommand") @@ -1061,6 +1063,7 @@ class PythonPipelineSuite gridTest("Unsupported SQL command inside query function should result in a failure")( unsupportedSqlCommandList) { unsupportedSqlCommand => + assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |@dp.materialized_view() From a49ffded2ad4a6b578b01e3d2d2bd1c2baeb3ca4 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 17 Nov 2025 08:25:10 -0800 Subject: [PATCH 155/400] [SPARK-54376][SDP] Mark most pipeline configuration options as internal ### What changes were proposed in this pull request? Marks all declarative pipelines configuration options as internal, except for `spark.sql.pipelines.maxFlowRetryAttempts`. ### Why are the changes needed? When implementing Declarative Pipelines, we made several quantities configurable. However, documented configurations are essentially public APIs, and it's too early to commit yet to supporting all of these. We should mark most of them internal except where we think users will really need them. ### Does this PR introduce _any_ user-facing change? Yes, to unreleased software. ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #53090 from sryza/internal-configs. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 1db267e3bd02003d2f88a97f67509642c50f6bd0) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3ec8366f8141e..cda75f22323ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6439,6 +6439,7 @@ object SQLConf { val PIPELINES_STREAM_STATE_POLLING_INTERVAL = { buildConf("spark.sql.pipelines.execution.streamstate.pollingInterval") + .internal() .doc( "Interval in seconds at which the stream state is polled for changes. This is used to " + "check if the stream has failed and needs to be restarted." @@ -6450,6 +6451,7 @@ object SQLConf { val PIPELINES_WATCHDOG_MIN_RETRY_TIME_IN_SECONDS = { buildConf("spark.sql.pipelines.execution.watchdog.minRetryTime") + .internal() .doc( "Initial duration in seconds between the time when we notice a flow has failed and " + "when we try to restart the flow. The interval between flow restarts doubles with " + @@ -6464,6 +6466,7 @@ object SQLConf { val PIPELINES_WATCHDOG_MAX_RETRY_TIME_IN_SECONDS = { buildConf("spark.sql.pipelines.execution.watchdog.maxRetryTime") + .internal() .doc( "Maximum time interval in seconds at which flows will be restarted." ) @@ -6474,6 +6477,7 @@ object SQLConf { val PIPELINES_MAX_CONCURRENT_FLOWS = { buildConf("spark.sql.pipelines.execution.maxConcurrentFlows") + .internal() .doc( "Max number of flows to execute at once. Used to tune performance for triggered " + "pipelines. Has no effect on continuous pipelines." @@ -6486,6 +6490,7 @@ object SQLConf { val PIPELINES_TIMEOUT_MS_FOR_TERMINATION_JOIN_AND_LOCK = { buildConf("spark.sql.pipelines.timeoutMsForTerminationJoinAndLock") + .internal() .doc("Timeout in milliseconds to grab a lock for stopping update - default is 1hr.") .version("4.1.0") .timeConf(TimeUnit.MILLISECONDS) @@ -6503,6 +6508,7 @@ object SQLConf { val PIPELINES_EVENT_QUEUE_CAPACITY = { buildConf("spark.sql.pipelines.event.queue.capacity") + .internal() .doc("Capacity of the event queue used in pipelined execution. When the queue is full, " + "non-terminal FlowProgressEvents will be dropped.") .version("4.1.0") From 82f70f3fb9e77d0dcca210316e8cb699f5fc4bb3 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Tue, 18 Nov 2025 06:48:49 +0900 Subject: [PATCH 156/400] [SPARK-54344][PYTHON] Kill the worker if flush fails in daemon.py ### What changes were proposed in this pull request? Kills the worker if flush fails in `daemon.py`. - Spark conf: `spark.python.daemon.killWorkerOnFlushFailure` (default `true`) - SQL conf: `spark.sql.execution.pyspark.udf.daemonKillWorkerOnFlushFailure` (fallback to the above) Before it just dies, reuse `faulthandler` feature and record the thread dump and it will appear in the error message if `faulthandler` is enabled. ``` WARN TaskSetManager: Lost task 3.0 in stage 1.0 (TID 8) (127.0.0.1 executor 1): org.apache.spark.SparkException: Python worker exited unexpectedly (crashed): Current thread 0x00000001f0796140 (most recent call first): File "/.../python/pyspark/daemon.py", line 95 in worker File "/.../python/pyspark/daemon.py", line 228 in manager File "/.../python/pyspark/daemon.py", line 253 in File "", line 88 in _run_code File "", line 198 in _run_module_as_main at org.apache.spark.api.python.BasePythonRunner$ReaderIterator$$anonfun$1.applyOrElse(PythonRunner.scala:679) ... ``` Even when `faulthandler` is not eabled, the error will appear in the executor's `stderr` file. ``` Traceback (most recent call last): File "/.../python/pyspark/daemon.py", line 228, in manager code = worker(sock, authenticated) ^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/.../python/pyspark/daemon.py", line 88, in worker raise Exception("test") Exception: test ``` When this is disabled, the behavior is the same as before but with a log. ### Why are the changes needed? Currently an exception caused by `outfile.flush()` failure in `daemon.py` is ignored, but if the last command in `worker_main` is still not flushed, it could cause a UDF stuck in Java waiting for the response from the Python worker. It should just die and let Spark retry the task. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually.

Test with the patch to emulate the case ```patch % git diff diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 54c9507e625..e107216d769 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py -84,6 +84,8 def worker(sock, authenticated): exit_code = compute_real_exit_code(exc.code) finally: try: + if worker_main.__globals__.get("TEST", False): + raise Exception("test") outfile.flush() except Exception: faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 6e34b041665..ff210f4fd97 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py -3413,7 +3413,14 def main(infile, outfile): # check end of stream if read_int(infile) == SpecialLengths.END_OF_STREAM: - write_int(SpecialLengths.END_OF_STREAM, outfile) + import random + + if random.random() < 0.1: + # emulate the last command is not flushed yet + global TEST + TEST = True + else: + write_int(SpecialLengths.END_OF_STREAM, outfile) else: # write a different value to tell JVM to not reuse this worker write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) -3423,6 +3430,9 def main(infile, outfile): faulthandler.cancel_dump_traceback_later() +TEST = False + + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. conn_info = os.environ.get( ```
With just `pass` (before this), it gets stuck, and after this it lets Spark retry the task. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53055 from ueshin/issues/SPARK-54344/daemon_flush. Lead-authored-by: Takuya Ueshin Co-authored-by: Takuya UESHIN Signed-off-by: Hyukjin Kwon (cherry picked from commit ed23cc39c32e0ca953f174f48ddff78ba24da375) Signed-off-by: Hyukjin Kwon --- .../apache/spark/api/python/PythonRunner.scala | 8 +++++++- .../org/apache/spark/internal/config/Python.scala | 12 ++++++++++++ python/pyspark/daemon.py | 15 ++++++++++++++- .../org/apache/spark/sql/internal/SQLConf.scala | 11 +++++++++++ .../sql/execution/python/ArrowPythonRunner.scala | 2 ++ .../execution/python/ArrowPythonUDTFRunner.scala | 2 ++ .../python/CoGroupedArrowPythonRunner.scala | 2 ++ .../execution/python/PythonPlannerRunner.scala | 4 ++++ .../sql/execution/python/PythonUDFRunner.scala | 2 ++ .../ApplyInPandasWithStatePythonRunner.scala | 2 ++ .../python/streaming/PythonForeachWriter.scala | 2 ++ 11 files changed, 60 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 66e204fee44b9..7f1dc7fc86fcd 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -188,12 +188,15 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( protected val timelyFlushEnabled: Boolean = false protected val timelyFlushTimeoutNanos: Long = 0 protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT) + private val useDaemon = conf.get(PYTHON_USE_DAEMON) private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) protected val faultHandlerEnabled: Boolean = conf.get(PYTHON_WORKER_FAULTHANLDER_ENABLED) protected val idleTimeoutSeconds: Long = conf.get(PYTHON_WORKER_IDLE_TIMEOUT_SECONDS) protected val killOnIdleTimeout: Boolean = conf.get(PYTHON_WORKER_KILL_ON_IDLE_TIMEOUT) protected val tracebackDumpIntervalSeconds: Long = conf.get(PYTHON_WORKER_TRACEBACK_DUMP_INTERVAL_SECONDS) + protected val killWorkerOnFlushFailure: Boolean = + conf.get(PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE) protected val hideTraceback: Boolean = false protected val simplifiedTraceback: Boolean = false @@ -294,13 +297,16 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (tracebackDumpIntervalSeconds > 0L) { envVars.put("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", tracebackDumpIntervalSeconds.toString) } + if (useDaemon && killWorkerOnFlushFailure) { + envVars.put("PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE", "1") + } // allow the user to set the batch size for the BatchedSerializer on UDFs envVars.put("PYTHON_UDF_BATCH_SIZE", batchSizeForPythonUDF.toString) envVars.put("SPARK_JOB_ARTIFACT_UUID", jobArtifactUUID.getOrElse("default")) val (worker: PythonWorker, handle: Option[ProcessHandle]) = env.createPythonWorker( - pythonExec, workerModule, daemonModule, envVars.asScala.toMap) + pythonExec, workerModule, daemonModule, envVars.asScala.toMap, useDaemon) // Whether is the worker released into idle pool or closed. When any codes try to release or // close a worker, they should use `releasedOrClosed.compareAndSet` to flip the state to make // sure there is only one winner that is going to release or close the worker. diff --git a/core/src/main/scala/org/apache/spark/internal/config/Python.scala b/core/src/main/scala/org/apache/spark/internal/config/Python.scala index de95e2fa1f7a2..dc16d1ff255db 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Python.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Python.scala @@ -138,4 +138,16 @@ private[spark] object Python { .intConf .checkValue(_ > 0, "If set, the idle worker max size must be > 0.") .createOptional + + val PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE = + ConfigBuilder("spark.python.daemon.killWorkerOnFlushFailure") + .doc("When enabled, exceptions raised during output flush operations in the Python " + + "worker managed under Python daemon are not caught, causing the worker to terminate " + + "with the exception. This allows Spark to detect the failure and launch a new worker " + + "and retry the task. " + + "When disabled, flush exceptions are caught and logged but the worker continues, " + + "which could cause the worker to get stuck due to protocol mismatch.") + .version("4.1.0") + .booleanConf + .createWithDefault(true) } diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index ca33ce2c39ef7..e75eca68fd0e7 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -24,6 +24,7 @@ import traceback import time import gc +import faulthandler from errno import EINTR, EAGAIN from socket import AF_INET, AF_INET6, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT @@ -85,7 +86,19 @@ def worker(sock, authenticated): try: outfile.flush() except Exception: - pass + if os.environ.get("PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE", False): + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + with open(faulthandler_log_path, "w") as faulthandler_log_file: + faulthandler.dump_traceback(file=faulthandler_log_file) + raise + else: + print( + "PySpark daemon failed to flush the output to the worker process:\n" + + traceback.format_exc(), + file=sys.stderr, + ) return exit_code diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index cda75f22323ee..f6aae1d2ba052 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3913,6 +3913,14 @@ object SQLConf { .version("4.1.0") .fallbackConf(Python.PYTHON_WORKER_TRACEBACK_DUMP_INTERVAL_SECONDS) + val PYTHON_UDF_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE = + buildConf("spark.sql.execution.pyspark.udf.daemonKillWorkerOnFlushFailure") + .doc( + s"Same as ${Python.PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE.key} " + + "for Python execution with DataFrame and SQL. It can change during runtime.") + .version("4.1.0") + .fallbackConf(Python.PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE) + val PYTHON_WORKER_LOGGING_ENABLED = buildConf("spark.sql.pyspark.worker.logging.enabled") .doc("When set to true, this configuration enables comprehensive logging within " + @@ -7410,6 +7418,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def pythonUDFWorkerTracebackDumpIntervalSeconds: Long = getConf(PYTHON_UDF_WORKER_TRACEBACK_DUMP_INTERVAL_SECONDS) + def pythonUDFDaemonKillWorkerOnFlushFailure: Boolean = + getConf(PYTHON_UDF_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE) + def pythonWorkerLoggingEnabled: Boolean = getConf(PYTHON_WORKER_LOGGING_ENABLED) def pythonUDFArrowConcurrencyLevel: Option[Int] = getConf(PYTHON_UDF_ARROW_CONCURRENCY_LEVEL) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index b94e00bc11ef2..f5f968ee95228 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -60,6 +60,8 @@ abstract class BaseArrowPythonRunner[IN, OUT <: AnyRef]( override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout override val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + override val killWorkerOnFlushFailure: Boolean = + SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure override val errorOnDuplicatedFieldNames: Boolean = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala index 7b73818bf0ec1..1d5df9bad9247 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala @@ -83,6 +83,8 @@ class ArrowPythonUDTFRunner( override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout override val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + override val killWorkerOnFlushFailure: Boolean = + SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure override val errorOnDuplicatedFieldNames: Boolean = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala index 50013e5338199..7f6efbae8881d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala @@ -71,6 +71,8 @@ class CoGroupedArrowPythonRunner( override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout override val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + override val killWorkerOnFlushFailure: Boolean = + SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure override val hideTraceback: Boolean = SQLConf.get.pysparkHideTraceback override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala index 0f4ac4ddad719..92e99cdc11d97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala @@ -58,6 +58,7 @@ abstract class PythonPlannerRunner[T](func: PythonFunction) extends Logging { val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + val killWorkerOnFlushFailure: Boolean = SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure val hideTraceback: Boolean = SQLConf.get.pysparkHideTraceback val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback val workerMemoryMb = SQLConf.get.pythonPlannerExecMemory @@ -98,6 +99,9 @@ abstract class PythonPlannerRunner[T](func: PythonFunction) extends Logging { if (tracebackDumpIntervalSeconds > 0L) { envVars.put("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", tracebackDumpIntervalSeconds.toString) } + if (useDaemon && killWorkerOnFlushFailure) { + envVars.put("PYTHON_DAEMON_KILL_WORKER_ON_FLUSH_FAILURE", "1") + } envVars.put("SPARK_JOB_ARTIFACT_UUID", jobArtifactUUID.getOrElse("default")) sessionUUID.foreach { uuid => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala index 61f493deeee49..759aa998832db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala @@ -59,6 +59,8 @@ abstract class BasePythonUDFRunner( override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout override val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + override val killWorkerOnFlushFailure: Boolean = + SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure override val bufferSize: Int = SQLConf.get.getConf(SQLConf.PYTHON_UDF_BUFFER_SIZE) override val batchSizeForPythonUDF: Int = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala index 51d9f6f523a23..14054ba89a948 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala @@ -79,6 +79,8 @@ class ApplyInPandasWithStatePythonRunner( override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout override val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + override val killWorkerOnFlushFailure: Boolean = + SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure private val sqlConf = SQLConf.get diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala index 37716d2d8413b..cc7745210a4d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala @@ -106,6 +106,8 @@ class PythonForeachWriter(func: PythonFunction, schema: StructType) override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout override val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds + override val killWorkerOnFlushFailure: Boolean = + SQLConf.get.pythonUDFDaemonKillWorkerOnFlushFailure override val hideTraceback: Boolean = SQLConf.get.pysparkHideTraceback override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback From 505d5f0d538dd62a0aaaf09ec408e670d2dc7b4f Mon Sep 17 00:00:00 2001 From: Kelvin Jiang Date: Tue, 18 Nov 2025 11:40:44 +0800 Subject: [PATCH 157/400] [SPARK-54339][SQL] Fix AttributeMap non-determinism ### What changes were proposed in this pull request? This PR fixes the `+`, `updated`, and `removed` methods of `AttributeMap` to correctly hash with `Attribute.ExprId` instead of `Attribute` as a whole. ### Why are the changes needed? This change fixes non-determinism with the `AttributeMap` when an entry is being added to the `AttributeMap` with `+` such that `attr1 != attr2` but `attr1.exprId = attr2.exprId`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added a new test suite. ### Was this patch authored or co-authored using generative AI tooling? Tests were generated by Claude Code on Sonnet 4.5. Closes #53044 from kelvinjian-db/fix-attributemap. Authored-by: Kelvin Jiang Signed-off-by: Wenchen Fan (cherry picked from commit 78d1d52601e4c43bc4c23f543ffb411416f1f6cd) Signed-off-by: Wenchen Fan --- .../catalyst/expressions/AttributeMap.scala | 6 +- .../expressions/AttributeMapSuite.scala | 278 ++++++++++++++++++ 2 files changed, 281 insertions(+), 3 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeMapSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index b317cacc061b7..9b6430c9ff0f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -48,14 +48,14 @@ class AttributeMap[A](val baseMap: Map[ExprId, (Attribute, A)]) override def contains(k: Attribute): Boolean = get(k).isDefined override def + [B1 >: A](kv: (Attribute, B1)): AttributeMap[B1] = - AttributeMap(baseMap.values.toMap + kv) + new AttributeMap(baseMap + (kv._1.exprId -> kv)) override def updated[B1 >: A](key: Attribute, value: B1): Map[Attribute, B1] = - baseMap.values.toMap + (key -> value) + this + (key -> value) override def iterator: Iterator[(Attribute, A)] = baseMap.valuesIterator - override def removed(key: Attribute): Map[Attribute, A] = baseMap.values.toMap - key + override def removed(key: Attribute): Map[Attribute, A] = new AttributeMap(baseMap - key.exprId) def ++(other: AttributeMap[A]): AttributeMap[A] = new AttributeMap(baseMap ++ other.baseMap) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeMapSuite.scala new file mode 100644 index 0000000000000..fbb37d452437b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeMapSuite.scala @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StringType} + +class AttributeMapSuite extends SparkFunSuite { + + val aUpper = AttributeReference("A", IntegerType)(exprId = ExprId(1)) + val aLower = AttributeReference("a", IntegerType)(exprId = ExprId(1)) + val fakeA = AttributeReference("a", IntegerType)(exprId = ExprId(3)) + + val bUpper = AttributeReference("B", IntegerType)(exprId = ExprId(2)) + val bLower = AttributeReference("b", IntegerType)(exprId = ExprId(2)) + + val cAttr = AttributeReference("c", StringType)(exprId = ExprId(4)) + + test("basic map operations - get") { + val map = AttributeMap(Seq((aUpper, "value1"), (bUpper, "value2"))) + + // Should find by exprId, not by attribute equality + assert(map.get(aLower) === Some("value1")) + assert(map.get(aUpper) === Some("value1")) + assert(map.get(bLower) === Some("value2")) + assert(map.get(bUpper) === Some("value2")) + + // Different exprId should not be found + assert(map.get(fakeA) === None) + } + + test("basic map operations - contains") { + val map = AttributeMap(Seq((aUpper, "value1"), (bUpper, "value2"))) + + // Should find by exprId, not by attribute equality + assert(map.contains(aLower)) + assert(map.contains(aUpper)) + assert(map.contains(bUpper)) + assert(!map.contains(fakeA)) + } + + test("basic map operations - getOrElse") { + val map = AttributeMap(Seq((aUpper, "value1"))) + + assert(map.getOrElse(aLower, "default") === "value1") + assert(map.getOrElse(fakeA, "default") === "default") + } + + test("+ operator preserves ExprId-based hashing") { + val map1 = AttributeMap(Seq((aUpper, "value1"))) + val map2 = map1 + (bUpper -> "value2") + + // The resulting map should still be an AttributeMap + assert(map2.isInstanceOf[AttributeMap[_]]) + + // Should look up by exprId, not by attribute equality + assert(map2.get(aLower) === Some("value1")) + assert(map2.get(bLower) === Some("value2")) + } + + test("+ operator with attribute having different metadata") { + val metadata1 = new MetadataBuilder().putString("key", "value1").build() + val metadata2 = new MetadataBuilder().putString("key", "value2").build() + + // Two attributes with same exprId but different metadata + val attrWithMetadata1 = AttributeReference("col", IntegerType, metadata = metadata1)( + exprId = ExprId(100)) + val attrWithMetadata2 = AttributeReference("col", IntegerType, metadata = metadata2)( + exprId = ExprId(100)) + + // These should have different hashCodes but same exprId + assert(attrWithMetadata1.hashCode() != attrWithMetadata2.hashCode(), + "Attributes with different metadata should have different hashCodes") + assert(attrWithMetadata1.exprId == attrWithMetadata2.exprId, + "Attributes should have the same exprId") + + // Create a map with the first attribute + val map1 = AttributeMap(Seq((attrWithMetadata1, "original"))) + + // Add an entry using the + operator + val map2 = map1 + (cAttr -> "new") + + // CRITICAL: The map should still find the original entry using an attribute + // with the same exprId but different metadata + assert(map2.get(attrWithMetadata2) === Some("original"), + "AttributeMap should look up by exprId, not by attribute hashCode") + + // And the new entry should also be present + assert(map2.get(cAttr) === Some("new")) + } + + test("+ operator updates existing key") { + val map1 = AttributeMap(Seq((aUpper, "value1"))) + val map2 = map1 + (aLower -> "updated") + + // Since aLower has the same exprId as aUpper, it should update the value + assert(map2.get(aUpper) === Some("updated")) + assert(map2.get(aLower) === Some("updated")) + assert(map2.size === 1) + } + + test("+ operator with type widening") { + val map1: AttributeMap[String] = AttributeMap(Seq((aUpper, "value1"))) + val map2: AttributeMap[Any] = map1 + (bUpper -> 42) + + assert(map2.get(aUpper) === Some("value1")) + assert(map2.get(bUpper) === Some(42)) + } + + test("++ operator preserves AttributeMap semantics") { + val map1 = AttributeMap(Seq((aUpper, "value1"))) + val map2 = AttributeMap(Seq((bUpper, "value2"))) + val combined = map1 ++ map2 + + assert(combined.isInstanceOf[AttributeMap[_]]) + assert(combined.get(aLower) === Some("value1")) + assert(combined.get(bLower) === Some("value2")) + } + + test("updated method") { + val map1 = AttributeMap(Seq((aUpper, "value1"))) + val map2 = map1.updated(bUpper, "value2") + + // Note: updated returns a Map[Attribute, B1], not AttributeMap + assert(map2.get(aUpper) === Some("value1")) + assert(map2.get(bUpper) === Some("value2")) + } + + test("- operator (removal)") { + val map1 = AttributeMap(Seq((aUpper, "value1"), (bUpper, "value2"))) + val map2 = map1 - aLower + + // Note: - returns a Map[Attribute, A], not AttributeMap + assert(map2.get(aUpper) === None) + assert(map2.get(bUpper) === Some("value2")) + } + + test("iterator") { + val map = AttributeMap(Seq((aUpper, "value1"), (bUpper, "value2"))) + val entries = map.iterator.toSeq + + assert(entries.size === 2) + assert(entries.contains((aUpper, "value1"))) + assert(entries.contains((bUpper, "value2"))) + } + + test("size") { + val emptyMap = AttributeMap.empty[String] + assert(emptyMap.size === 0) + + val map1 = AttributeMap(Seq((aUpper, "value1"))) + assert(map1.size === 1) + + val map2 = AttributeMap(Seq((aUpper, "value1"), (bUpper, "value2"))) + assert(map2.size === 2) + } + + test("empty map") { + val emptyMap = AttributeMap.empty[String] + assert(emptyMap.get(aUpper) === None) + assert(emptyMap.size === 0) + assert(!emptyMap.contains(aUpper)) + } + + test("duplicate keys in construction") { + // When constructing with duplicate exprIds, the last one should win + val map = AttributeMap(Seq((aUpper, "value1"), (aLower, "value2"))) + assert(map.size === 1) + assert(map.get(aUpper) === Some("value2")) + } + + test("map construction from Map") { + val regularMap = Map(aUpper -> "value1", bUpper -> "value2") + val attrMap = AttributeMap(regularMap) + + assert(attrMap.get(aLower) === Some("value1")) + assert(attrMap.get(bLower) === Some("value2")) + } + + test("chained + operations") { + val map = AttributeMap.empty[String] + (aUpper -> "value1") + (bUpper -> "value2") + + (cAttr -> "value3") + + assert(map.size === 3) + assert(map.get(aLower) === Some("value1")) + assert(map.get(bLower) === Some("value2")) + assert(map.get(cAttr) === Some("value3")) + } + + test("+ should be deterministic with Attributes with diff hashcodes and same exprId") { + // The HashMap needs to be of a certain size before the hashing starts to collide, set up + // these AttributeMaps to start with size 5. + var map1 = AttributeMap( + Seq( + AttributeReference("a", IntegerType)(exprId = ExprId(1)) -> 1, + AttributeReference("b", IntegerType)(exprId = ExprId(2)) -> 2, + AttributeReference("c", IntegerType)(exprId = ExprId(3)) -> 3, + AttributeReference("d", IntegerType)(exprId = ExprId(4)) -> 4, + AttributeReference("e", IntegerType)(exprId = ExprId(5)) -> 5 + ) + ) + var map2 = AttributeMap( + Seq( + AttributeReference("a", IntegerType)(exprId = ExprId(1)) -> 1, + AttributeReference("b", IntegerType)(exprId = ExprId(2)) -> 2, + AttributeReference("c", IntegerType)(exprId = ExprId(3)) -> 3, + AttributeReference("d", IntegerType)(exprId = ExprId(4)) -> 4, + AttributeReference("e", IntegerType)(exprId = ExprId(5)) -> 5 + ) + ) + val qualifier1 = Seq("d") + val qualifier2 = Seq() + val exprId = ExprId(42) + val attr1 = AttributeReference("x", IntegerType)(exprId = exprId, qualifier = qualifier1) + val attr2 = AttributeReference("x", IntegerType)(exprId = exprId, qualifier = qualifier2) + assert(attr1.hashCode != attr2.hashCode) + + map1 = map1 + (attr1 -> 100) + map1 = map1 + (attr2 -> 200) + assert(map1.get(attr2) === Some(200)) + + map2 = map2 + (attr2 -> 200) + map2 = map2 + (attr1 -> 100) + assert(map2.get(attr2) === Some(100)) + } + + test("updated should be deterministic with Attributes with diff hashcodes and same exprId") { + // The HashMap needs to be of a certain size before the hashing starts to collide, set up + // these AttributeMaps to start with size 5. + var map1: Map[Attribute, Int] = AttributeMap( + Seq( + AttributeReference("a", IntegerType)(exprId = ExprId(1)) -> 1, + AttributeReference("b", IntegerType)(exprId = ExprId(2)) -> 2, + AttributeReference("c", IntegerType)(exprId = ExprId(3)) -> 3, + AttributeReference("d", IntegerType)(exprId = ExprId(4)) -> 4, + AttributeReference("e", IntegerType)(exprId = ExprId(5)) -> 5 + ) + ) + var map2: Map[Attribute, Int] = AttributeMap( + Seq( + AttributeReference("a", IntegerType)(exprId = ExprId(1)) -> 1, + AttributeReference("b", IntegerType)(exprId = ExprId(2)) -> 2, + AttributeReference("c", IntegerType)(exprId = ExprId(3)) -> 3, + AttributeReference("d", IntegerType)(exprId = ExprId(4)) -> 4, + AttributeReference("e", IntegerType)(exprId = ExprId(5)) -> 5 + ) + ) + val qualifier1 = Seq("d") + val qualifier2 = Seq() + val exprId = ExprId(42) + val attr1 = AttributeReference("x", IntegerType)(exprId = exprId, qualifier = qualifier1) + val attr2 = AttributeReference("x", IntegerType)(exprId = exprId, qualifier = qualifier2) + assert(attr1.hashCode != attr2.hashCode) + + map1 = map1.updated(attr1, 100) + map1 = map1.updated(attr2, 200) + assert(map1.get(attr2) === Some(200)) + + map2 = map2.updated(attr2, 200) + map2 = map2.updated(attr1, 100) + assert(map2.get(attr2) === Some(100)) + } +} From 6e78983ecbdd0a0f93f6cbaae86718a73664638d Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 17 Nov 2025 22:08:53 -0800 Subject: [PATCH 158/400] [SPARK-53924][FOLLOWUP][TESTS] Add tests for cached temp view detecting schema changes ### What changes were proposed in this pull request? Follow-up of https://github.com/apache/spark/pull/52876, add tests for cached temp view detecting schema changes ### Why are the changes needed? There is no test coverage after comment https://github.com/apache/spark/pull/52876#discussion_r2529313684 is addressed. This PR is to add a test case for it. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New test case ### Was this patch authored or co-authored using generative AI tooling? No Closes #53103 from gengliangwang/SPARK-53924-test. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang (cherry picked from commit fd683ce6dbd3eb376fb3618dba55e711606876c1) Signed-off-by: Gengliang Wang --- .../DataSourceV2DataFrameSuite.scala | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index c59e624cb1784..d802a9a7d75f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1173,6 +1173,34 @@ class DataSourceV2DataFrameSuite } } + test("SPARK-54157: cached temp view detects schema changes after analysis") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, data STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a')") + + // create a temp view on top of the DSv2 table and cache the view + spark.table(t).createOrReplaceTempView("v") + sql("CACHE TABLE v") + assertCached(sql("SELECT * FROM v")) + + // change table schema after the view has been analyzed and cached + sql(s"ALTER TABLE $t ADD COLUMN extra INT") + + // execution should fail with column mismatch even though the view is cached + checkError( + exception = intercept[AnalysisException] { spark.table("v").collect() }, + condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", + parameters = Map( + "viewName" -> "`v`", + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "colType" -> "data", + "errors" -> + """ + |- `extra` INT has been added""".stripMargin)) + } + } + test("SPARK-54157: detect nested struct field changes after DataFrame analysis") { val t = "testcat.ns1.ns2.tbl" withTable(t) { From 1185db494dda3bdff350049d40a938c19b02c7b8 Mon Sep 17 00:00:00 2001 From: Ganesha S Date: Tue, 18 Nov 2025 16:42:05 +0800 Subject: [PATCH 159/400] [SPARK-54377][SQL] Fix COMMENT ON TABLE IS NULL to properly remove table comment ### What changes were proposed in this pull request? This PR fixes a bug where COMMENT ON TABLE table_name IS NULL was not properly removing the table comment. ### Why are the changes needed? The syntax COMMENT ON TABLE table_name IS NULL should remove the table comment. However, the previous implementation was setting the comment to null rather than removing the property entirely. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Enhanced test case test("COMMENT ON TABLE") in DataSourceV2SQLSuite verifies: * Comment can be set and is stored correctly * Comment is completely removed when set to NULL (property no longer exists) * Literal string "NULL" can still be set as a comment value * Works for both session catalog and V2 catalogs ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.5 Closes #53091 from ganeshashree/SPARK-54377. Authored-by: Ganesha S Signed-off-by: Wenchen Fan (cherry picked from commit e8f0a67e248d67e5e1951015a378c36282d12e17) Signed-off-by: Wenchen Fan --- .../plans/logical/v2AlterTableCommands.scala | 6 +- .../sql/connector/DataSourceV2SQLSuite.scala | 71 +++++++++++++++++-- 2 files changed, 72 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index d1eb561f3add1..4ec8baf351cb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -48,7 +48,11 @@ trait AlterTableCommand extends UnaryCommand { */ case class CommentOnTable(table: LogicalPlan, comment: String) extends AlterTableCommand { override def changes: Seq[TableChange] = { - Seq(TableChange.setProperty(TableCatalog.PROP_COMMENT, comment)) + if (comment == null) { + Seq(TableChange.removeProperty(TableCatalog.PROP_COMMENT)) + } else { + Seq(TableChange.setProperty(TableCatalog.PROP_COMMENT, comment)) + } } override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = copy(table = newChild) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 907820e462380..847af570d6f39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2737,8 +2737,35 @@ class DataSourceV2SQLSuiteV1Filter // Session catalog is used. withTable("t") { sql("CREATE TABLE t(k int) USING json") + + // Verify no comment initially + val noCommentRows1 = sql("DESC EXTENDED t").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(noCommentRows1.isEmpty || noCommentRows1.head.getString(1).isEmpty, + "Expected no comment initially") + + // Set a comment checkTableComment("t", "minor revision") + + // Verify comment is set + val commentRows1 = sql("DESC EXTENDED t").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(commentRows1.nonEmpty && commentRows1.head.getString(1) === "minor revision", + "Expected comment to be set to 'minor revision'") + + // Remove comment by setting to NULL checkTableComment("t", null) + + // Verify comment is removed + val removedCommentRows1 = sql("DESC EXTENDED t").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(removedCommentRows1.isEmpty || removedCommentRows1.head.getString(1).isEmpty, + "Expected comment to be removed") + + // Set comment to literal "NULL" string checkTableComment("t", "NULL") } val sql1 = "COMMENT ON TABLE abc IS NULL" @@ -2751,8 +2778,35 @@ class DataSourceV2SQLSuiteV1Filter // V2 non-session catalog is used. withTable("testcat.ns1.ns2.t") { sql("CREATE TABLE testcat.ns1.ns2.t(k int) USING foo") + + // Verify no comment initially + val noCommentRows2 = sql("DESC EXTENDED testcat.ns1.ns2.t").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(noCommentRows2.isEmpty || noCommentRows2.head.getString(1).isEmpty, + "Expected no comment initially for testcat table") + + // Set a comment checkTableComment("testcat.ns1.ns2.t", "minor revision") + + // Verify comment is set + val commentRows2 = sql("DESC EXTENDED testcat.ns1.ns2.t").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(commentRows2.nonEmpty && commentRows2.head.getString(1) === "minor revision", + "Expected comment to be set to 'minor revision' for testcat table") + + // Remove comment by setting to NULL checkTableComment("testcat.ns1.ns2.t", null) + + // Verify comment is removed + val removedCommentRows2 = sql("DESC EXTENDED testcat.ns1.ns2.t").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(removedCommentRows2.isEmpty || removedCommentRows2.head.getString(1).isEmpty, + "Expected comment to be removed from testcat table") + + // Set comment to literal "NULL" string checkTableComment("testcat.ns1.ns2.t", "NULL") } val sql2 = "COMMENT ON TABLE testcat.abc IS NULL" @@ -2778,10 +2832,19 @@ class DataSourceV2SQLSuiteV1Filter private def checkTableComment(tableName: String, comment: String): Unit = { sql(s"COMMENT ON TABLE $tableName IS " + Option(comment).map("'" + _ + "'").getOrElse("NULL")) - val expectedComment = Option(comment).getOrElse("") - assert(sql(s"DESC extended $tableName").toDF("k", "v", "c") - .where(s"k='${TableCatalog.PROP_COMMENT.capitalize}'") - .head().getString(1) === expectedComment) + if (comment == null) { + // When comment is NULL, the property should be removed completely + val commentRows = sql(s"DESC extended $tableName").toDF("k", "v", "c") + .where("k='Comment'") + .collect() + assert(commentRows.isEmpty || commentRows.head.getString(1).isEmpty, + "Expected comment to be removed") + } else { + val expectedComment = comment + assert(sql(s"DESC extended $tableName").toDF("k", "v", "c") + .where("k='Comment'") + .head().getString(1) === expectedComment) + } } test("SPARK-31015: star expression should work for qualified column names for v2 tables") { From 7d932008599927797f7e902ed10abc466675c331 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Tue, 18 Nov 2025 22:35:43 +0800 Subject: [PATCH 160/400] [SPARK-52767][SQL] Optimize maxRows and maxRowsPerPartition for join and union ### What changes were proposed in this pull request? Make the `maxRows` and `maxRowsPerPartition` only calculated at most once. ### Why are the changes needed? Improve performance, especially when there are dozens of joins and unions. Before pr, the number of maxRows executions of join/union increases exponentially with the number of joins/unions. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Local test, 28 tables join before pr 36s, after pr 4s, 29 tables join before pr 67s, after pr 5s ``` Seq(1).toDF("a").write.mode("overwrite").parquet("tmp/t1") spark.read.parquet("tmp/t1").createOrReplaceTempView("t") val t1 = System.currentTimeMillis() spark.sql( """ |select a,count(1) from ( |select t1.a from (select distinct a from t) t1 |join t t2 on t1.a=t2.a |join t t3 on t1.a=t3.a |join t t4 on t1.a=t4.a |join t t5 on t1.a=t5.a |join t t6 on t1.a=t6.a |join t t7 on t1.a=t7.a |join t t8 on t1.a=t8.a |join t t9 on t1.a=t9.a |join t t10 on t1.a=t10.a |join t t11 on t1.a=t11.a |join t t12 on t1.a=t12.a |join t t13 on t1.a=t13.a |join t t14 on t1.a=t14.a |join t t15 on t1.a=t15.a |join t t16 on t1.a=t16.a |join t t17 on t1.a=t17.a |join t t18 on t1.a=t18.a |join t t19 on t1.a=t19.a |join t t20 on t1.a=t20.a |join t t21 on t1.a=t21.a |join t t22 on t1.a=t22.a |join t t23 on t1.a=t23.a |join t t24 on t1.a=t24.a |join t t25 on t1.a=t25.a |join t t26 on t1.a=t26.a |join t t27 on t1.a=t27.a |join t t28 on t1.a=t28.a |) group by a |""".stripMargin).show ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #51451 from zml1206/SPARK-52767. Authored-by: zml1206 Signed-off-by: Wenchen Fan (cherry picked from commit aa387f32158a98260f7b9b16dc87feb64b504ab4) Signed-off-by: Wenchen Fan --- .../plans/logical/basicLogicalOperators.scala | 44 +++++++++---------- 1 file changed, 21 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 142420ee258ae..b87d018f2ab1d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -580,19 +580,18 @@ case class Union( allowMissingCol: Boolean = false) extends UnionBase { assert(!allowMissingCol || byName, "`allowMissingCol` can be true only if `byName` is true.") - override def maxRows: Option[Long] = { - var sum = BigInt(0) - children.foreach { child => - if (child.maxRows.isDefined) { - sum += child.maxRows.get - if (!sum.isValidLong) { - return None + override lazy val maxRows: Option[Long] = { + val sum = children.foldLeft(Option(BigInt(0))) { + case (Some(acc), child) => + child.maxRows match { + case Some(n) => + val newSum = acc + n + if (newSum.isValidLong) Some(newSum) else None + case None => None } - } else { - return None - } + case (None, _) => None } - Some(sum.toLong) + sum.map(_.toLong) } final override val nodePatterns: Seq[TreePattern] = Seq(UNION) @@ -600,19 +599,18 @@ case class Union( /** * Note the definition has assumption about how union is implemented physically. */ - override def maxRowsPerPartition: Option[Long] = { - var sum = BigInt(0) - children.foreach { child => - if (child.maxRowsPerPartition.isDefined) { - sum += child.maxRowsPerPartition.get - if (!sum.isValidLong) { - return None + override lazy val maxRowsPerPartition: Option[Long] = { + val sum = children.foldLeft(Option(BigInt(0))) { + case (Some(acc), child) => + child.maxRowsPerPartition match { + case Some(n) => + val newSum = acc + n + if (newSum.isValidLong) Some(newSum) else None + case None => None } - } else { - return None - } + case (None, _) => None } - Some(sum.toLong) + sum.map(_.toLong) } private def duplicatesResolvedPerBranch: Boolean = @@ -666,7 +664,7 @@ case class Join( hint: JoinHint) extends BinaryNode with PredicateHelper { - override def maxRows: Option[Long] = { + override lazy val maxRows: Option[Long] = { joinType match { case Inner | Cross | FullOuter | LeftOuter | RightOuter | LeftSingle if left.maxRows.isDefined && right.maxRows.isDefined => From 9306a76d4702c29660c5da2a148669cca77b9701 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 18 Nov 2025 08:05:45 -0800 Subject: [PATCH 161/400] [SPARK-54387][SQL] Fix recaching of DSv2 tables ### What changes were proposed in this pull request? This PR fixes recaching of DSv2 tables. ### Why are the changes needed? These changes are needed to restore correct caching behavior for DSv2 tables if a connector doesn't reuse table instances. Currently, the following use case is broken: ``` // create and populate table sql("CREATE TABLE testcat.ns.tbl (id bigint, data string) USING foo") Seq((1L, "a"), (2L, "b")).toDF("id", "data").write.insertInto("testcat.ns.tbl") // cache table val df1 = spark.table("testcat.ns.tbl") df1.cache() df1.show() // 1 -> a, 2 -> b // insert more data, refreshing cache entry Seq((3L, "c"), (4L, "d")).toDF("id", "data").write.insertInto("testcat.ns.tbl") // query val df2 = spark.table("testcat.ns.tbl") df2.show() // CACHE MISS BEFORE CHANGE! ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing + new tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53109 from aokolnychyi/spark-54387. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun (cherry picked from commit dce992b7708bc0d0f365aa94a40991a225696145) Signed-off-by: Dongjoon Hyun --- .../catalyst/plans/logical/v2Commands.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 2 ++ .../datasources/v2/V2TableRefreshUtil.scala | 12 ++++++---- .../spark/sql/execution/CacheManager.scala | 10 ++++---- .../spark/sql/execution/QueryExecution.scala | 2 +- .../execution/columnar/InMemoryRelation.scala | 7 +++--- .../apache/spark/sql/CachedTableSuite.scala | 1 + .../DataSourceV2DataFrameSuite.scala | 24 +++++++++++++++++++ 8 files changed, 47 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index c9d6fb61ba69f..614b73b1547f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -692,7 +692,7 @@ case class ReplaceTableAsSelect( // RTAS may drop and recreate table before query execution, breaking self-references // refresh and pin versions here to read from original table versions instead of // newly created empty table that is meant to serve as target for append/overwrite - val refreshedQuery = V2TableRefreshUtil.refreshVersions(query) + val refreshedQuery = V2TableRefreshUtil.refresh(query, versionedOnly = true) val pinnedQuery = V2TableRefreshUtil.pinVersions(refreshedQuery) copy(query = pinnedQuery, isAnalyzed = true) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 3462ae0e4206e..b0fb414fce976 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -133,6 +133,8 @@ case class DataSourceV2Relation( def autoSchemaEvolution(): Boolean = table.capabilities().contains(TableCapability.AUTOMATIC_SCHEMA_EVOLUTION) + + def isVersioned: Boolean = table.currentVersion != null } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala index 4a21416292406..e98b80b6a5a0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala @@ -40,7 +40,7 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { def pinVersions(plan: LogicalPlan): LogicalPlan = { plan transform { case r @ ExtractV2CatalogAndIdentifier(catalog, ident) - if r.table.currentVersion != null && r.timeTravelSpec.isEmpty => + if r.isVersioned && r.timeTravelSpec.isEmpty => val tableName = V2TableUtil.toQualifiedName(catalog, ident) val version = r.table.currentVersion logDebug(s"Pinning table version for $tableName to $version") @@ -49,21 +49,25 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { } /** - * Refreshes table metadata for all versioned tables in the plan. + * Refreshes table metadata for tables in the plan. * * This method reloads table metadata from the catalog and validates: * - Table identity: Ensures table ID has not changed * - Data columns: Verifies captured columns match the current schema * - Metadata columns: Checks metadata column consistency * + * Tables with time travel specifications are skipped as they reference a specific point + * in time and don't have to be refreshed. + * * @param plan the logical plan to refresh + * @param versionedOnly indicates whether to refresh only versioned tables * @return plan with refreshed table metadata */ - def refreshVersions(plan: LogicalPlan): LogicalPlan = { + def refresh(plan: LogicalPlan, versionedOnly: Boolean = false): LogicalPlan = { val cache = mutable.HashMap.empty[(TableCatalog, Identifier), Table] plan transform { case r @ ExtractV2CatalogAndIdentifier(catalog, ident) - if r.table.currentVersion != null && r.timeTravelSpec.isEmpty => + if (r.isVersioned || !versionedOnly) && r.timeTravelSpec.isEmpty => val currentTable = cache.getOrElseUpdate((catalog, ident), { val tableName = V2TableUtil.toQualifiedName(catalog, ident) logDebug(s"Refreshing table metadata for $tableName") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 3944cf818895e..a35efd96060f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation, LogicalRelationWithTable} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, ExtractV2Table, FileTable} +import org.apache.spark.sql.execution.datasources.v2.V2TableRefreshUtil import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -352,11 +353,12 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { needToRecache.foreach { cd => cd.cachedRepresentation.cacheBuilder.clearCache() val sessionWithConfigsOff = getOrCloneSessionWithConfigsOff(spark) - val newCache = sessionWithConfigsOff.withActive { - val qe = sessionWithConfigsOff.sessionState.executePlan(cd.plan) - InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) + val (newKey, newCache) = sessionWithConfigsOff.withActive { + val refreshedPlan = V2TableRefreshUtil.refresh(cd.plan) + val qe = sessionWithConfigsOff.sessionState.executePlan(refreshedPlan) + qe.normalized -> InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) } - val recomputedPlan = cd.copy(cachedRepresentation = newCache) + val recomputedPlan = cd.copy(plan = newKey, cachedRepresentation = newCache) this.synchronized { if (lookupCachedDataInternal(recomputedPlan.plan).nonEmpty) { logWarning("While recaching, data was already added to cache.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 5c7fbcc8edd04..12fce2f91dac8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -207,7 +207,7 @@ class QueryExecution( // there may be delay between analysis and subsequent phases // therefore, refresh captured table versions to reflect latest data private val lazyTableVersionsRefreshed = LazyTry { - V2TableRefreshUtil.refreshVersions(commandExecuted) + V2TableRefreshUtil.refresh(commandExecuted, versionedOnly = true) } private[sql] def tableVersionsRefreshed: LogicalPlan = lazyTableVersionsRefreshed.get diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index bf7491625fa03..56faf2032065d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -406,11 +406,12 @@ object InMemoryRelation { def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = { val optimizedPlan = qe.optimizedPlan val serializer = cacheBuilder.serializer - val newBuilder = if (serializer.supportsColumnarInput(optimizedPlan.output)) { - cacheBuilder.copy(cachedPlan = serializer.convertToColumnarPlanIfPossible(qe.executedPlan)) + val newCachedPlan = if (serializer.supportsColumnarInput(optimizedPlan.output)) { + serializer.convertToColumnarPlanIfPossible(qe.executedPlan) } else { - cacheBuilder.copy(cachedPlan = qe.executedPlan) + qe.executedPlan } + val newBuilder = cacheBuilder.copy(cachedPlan = newCachedPlan, logicalPlan = qe.logical) val relation = new InMemoryRelation( newBuilder.cachedPlan.output, newBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 91addd72ab2b2..7faf580b6f7f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -68,6 +68,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils override def sparkConf: SparkConf = super.sparkConf .set("spark.sql.catalog.testcat", classOf[InMemoryCatalog].getName) + .set("spark.sql.catalog.testcat.copyOnLoad", "true") setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index d802a9a7d75f8..205fa561a5b00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1620,6 +1620,30 @@ class DataSourceV2DataFrameSuite } } + test("cached DSv2 table DataFrame is refreshed and reused after insert") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + val df1 = Seq((1L, "a"), (2L, "b")).toDF("id", "data") + df1.write.insertInto(t) + + // cache DataFrame pointing to table + val readDF1 = spark.table(t) + readDF1.cache() + assertCached(readDF1) + checkAnswer(readDF1, Seq(Row(1L, "a"), Row(2L, "b"))) + + // insert more data, invalidating and refreshing cache entry + val df2 = Seq((3L, "c"), (4L, "d")).toDF("id", "data") + df2.write.insertInto(t) + + // verify underlying plan is recached and picks up new data + val readDF2 = spark.table(t) + assertCached(readDF2) + checkAnswer(readDF2, Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"), Row(4L, "d"))) + } + } + private def pinTable(catalogName: String, ident: Identifier, version: String): Unit = { catalog(catalogName) match { case inMemory: BasicInMemoryTableCatalog => inMemory.pinTable(ident, version) From 9bc3e1b0bf1b419abd242212e2c10adda6e2a140 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 18 Nov 2025 10:44:35 -0800 Subject: [PATCH 162/400] [SPARK-54350][SQL][STS] SparkGetColumnsOperation ORDINAL_POSITION should be 1-based ### What changes were proposed in this pull request? The SparkGetColumnsOperation is mainly used for the JDBC driver, while JDBC uses 1-based ordinal/column-index instead of 0-based. This is also documented in Hive API. https://github.com/apache/spark/blob/551b922a53acfdfeb2c065d5dedf35cb8cd30e1d/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java#L94-L95 Note, the GetColumnsOperation, which is originally copied from the Hive has a correct implementation, the issue only exists in SparkGetColumnsOperation. For safety, a config `spark.sql.legacy.hive.thriftServer.useZeroBasedColumnOrdinalPosition` is added to allow the user to switch back to the previous behavior. ### Why are the changes needed? The SparkGetColumnsOperation is mainly used by JDBC [java.sql.DatabaseMetaData#getColumns](https://docs.oracle.com/en/java/javase/17/docs/api/java.sql/java/sql/DatabaseMetaData.html#getColumns(java.lang.String,java.lang.String,java.lang.String,java.lang.String)), this change makes it satisfy the JDBC API specification. ### Does this PR introduce _any_ user-facing change? Yes, see the above section. ### How was this patch tested? UTs are modified. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53062 from pan3793/SPARK-54350. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 05bc5d408fa9e34049f64a75825e3d414482791a) Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 1 + .../SparkGetColumnsOperation.scala | 8 +++- .../SparkMetadataOperationSuite.scala | 40 ++++++++++++++++--- .../org/apache/spark/sql/hive/HiveUtils.scala | 8 ++++ 4 files changed, 50 insertions(+), 7 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index e5becac540328..0a2533d28f0b8 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -25,6 +25,7 @@ license: | ## Upgrading from Spark SQL 4.0 to 4.1 - Since Spark 4.1, the Parquet reader no longer assumes all struct values to be null, if all the requested fields are missing in the parquet file. The new default behavior is to read an additional struct field that is present in the file to determine nullness. To restore the previous behavior, set `spark.sql.legacy.parquet.returnNullStructIfAllFieldsMissing` to `true`. +- Since Spark 4.1, the Spark Thrift Server returns the corrected 1-based ORDINAL_POSITION in the result of GetColumns operation, instead of the wrongly 0-based. To restore the previous behavior, set `spark.sql.legacy.hive.thriftServer.useZeroBasedColumnOrdinalPosition` to `true`. ## Upgrading from Spark SQL 3.5 to 4.0 diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 6c573ceb14ecf..bdfd84e9da5fb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.types._ /** @@ -200,6 +201,11 @@ private[hive] class SparkGetColumnsOperation( schema.zipWithIndex.foreach { case (column, pos) => if (columnPattern != null && !columnPattern.matcher(column.name).matches()) { } else { + val ordinal = if (session.conf.get(HiveUtils.LEGACY_STS_ZERO_BASED_COLUMN_ORDINAL)) { + pos + } else { + pos + 1 + } val rowData = Array[AnyRef]( null, // TABLE_CAT dbName, // TABLE_SCHEM @@ -217,7 +223,7 @@ private[hive] class SparkGetColumnsOperation( null, // SQL_DATA_TYPE null, // SQL_DATETIME_SUB null, // CHAR_OCTET_LENGTH - pos.asInstanceOf[AnyRef], // ORDINAL_POSITION + ordinal.asInstanceOf[AnyRef], // ORDINAL_POSITION, 1-based "YES", // IS_NULLABLE null, // SCOPE_CATALOG null, // SCOPE_SCHEMA diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index abd2b1983b34e..a10d2974db744 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -24,6 +24,7 @@ import org.apache.hive.service.cli.HiveSQLException import org.apache.spark.SPARK_VERSION import org.apache.spark.sql.catalyst.analysis.FunctionRegistry +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.VersionUtils @@ -341,7 +342,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(rowSet.getInt("NULLABLE") === 1) assert(rowSet.getString("REMARKS") === pos.toString) - assert(rowSet.getInt("ORDINAL_POSITION") === pos) + assert(rowSet.getInt("ORDINAL_POSITION") === pos + 1) assert(rowSet.getString("IS_NULLABLE") === "YES") assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") pos += 1 @@ -372,7 +373,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(rowSet.getInt("NUM_PREC_RADIX") === 0) assert(rowSet.getInt("NULLABLE") === 0) assert(rowSet.getString("REMARKS") === "") - assert(rowSet.getInt("ORDINAL_POSITION") === 0) + assert(rowSet.getInt("ORDINAL_POSITION") === 1) assert(rowSet.getString("IS_NULLABLE") === "YES") assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") } @@ -400,7 +401,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(rowSet.getInt("NUM_PREC_RADIX") === 0) assert(rowSet.getInt("NULLABLE") === 0) assert(rowSet.getString("REMARKS") === "") - assert(rowSet.getInt("ORDINAL_POSITION") === 0) + assert(rowSet.getInt("ORDINAL_POSITION") === 1) assert(rowSet.getString("IS_NULLABLE") === "YES") assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") } @@ -426,7 +427,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(rowSet.getInt("NUM_PREC_RADIX") === 0) assert(rowSet.getInt("NULLABLE") === 0) assert(rowSet.getString("REMARKS") === "") - assert(rowSet.getInt("ORDINAL_POSITION") === 0) + assert(rowSet.getInt("ORDINAL_POSITION") === 1) assert(rowSet.getString("IS_NULLABLE") === "YES") assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") } @@ -453,7 +454,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(rowSet.getInt("NUM_PREC_RADIX") === 0) assert(rowSet.getInt("NULLABLE") === 1) assert(rowSet.getString("REMARKS") === "") - assert(rowSet.getInt("ORDINAL_POSITION") === 0) + assert(rowSet.getInt("ORDINAL_POSITION") === 1) assert(rowSet.getString("IS_NULLABLE") === "YES") assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") } @@ -680,9 +681,36 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(rowSet.getInt("DECIMAL_DIGITS") === 6) assert(rowSet.getInt("NUM_PREC_RADIX") === 0) assert(rowSet.getInt("NULLABLE") === 0) - assert(rowSet.getInt("ORDINAL_POSITION") === idx) + assert(rowSet.getInt("ORDINAL_POSITION") === idx + 1) idx += 1 } } } + + test("SPARK-54350: SparkGetColumnsOperation respects useZeroBasedColumnOrdinalPosition config") { + Seq(true, false).foreach { zeroBasedOrdinal => + val viewName = "view_column_ordinal_position" + val ddl = s"CREATE OR REPLACE GLOBAL TEMPORARY VIEW $viewName AS " + + "SELECT 1 AS id, 'foo' AS name" + + withJdbcStatement(viewName) { statement => + statement.execute( + s"SET ${HiveUtils.LEGACY_STS_ZERO_BASED_COLUMN_ORDINAL.key}=$zeroBasedOrdinal") + statement.execute(ddl) + val data = statement.getConnection.getMetaData + val rowSet = data.getColumns("", "global_temp", viewName, null) + assert(rowSet.next()) + assert(rowSet.getString("TABLE_SCHEM") === "global_temp") + assert(rowSet.getString("TABLE_NAME") === viewName) + assert(rowSet.getString("COLUMN_NAME") === "id") + assert(rowSet.getInt("ORDINAL_POSITION") === (if (zeroBasedOrdinal) 0 else 1)) + assert(rowSet.next()) + assert(rowSet.getString("TABLE_SCHEM") === "global_temp") + assert(rowSet.getString("TABLE_NAME") === viewName) + assert(rowSet.getString("COLUMN_NAME") === "name") + assert(rowSet.getInt("ORDINAL_POSITION") === (if (zeroBasedOrdinal) 1 else 2)) + assert(!rowSet.next()) + } + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index ac346a5b3ecf4..f0e1e208e5426 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -222,6 +222,14 @@ private[spark] object HiveUtils extends Logging { .booleanConf .createWithDefault(true) + val LEGACY_STS_ZERO_BASED_COLUMN_ORDINAL = + buildConf("spark.sql.legacy.hive.thriftServer.useZeroBasedColumnOrdinalPosition") + .doc("When set to true, Hive Thrift server returns 0-based ORDINAL_POSITION in the " + + "result of GetColumns operation, instead of the corrected 1-based.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + val USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT = buildConf("spark.sql.hive.useDelegateForSymlinkTextInputFormat") .internal() From 2965edb4df1aaa282e05110121c889f8426cf1a7 Mon Sep 17 00:00:00 2001 From: yhuang-db Date: Tue, 18 Nov 2025 10:51:58 -0800 Subject: [PATCH 163/400] [SPARK-54163][SQL] Scan canonicalization for partitioning and ordering info ### What changes were proposed in this pull request? This PR extends current canonicalization function for DataSourceV2ScanRelation to normalize the keyGroupedPartitioning and ordering field. Therefore it can apply to partition/ordering-aware data sources. ### Why are the changes needed? In order to apply canonicalization to partition/ordering-aware data sources. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53105 from yhuang-db/SPARK-54163-canonicalization-normalization. Authored-by: yhuang-db Signed-off-by: Gengliang Wang (cherry picked from commit 1012a5ffa51b09b06052e27f51a9c99dcfea4337) Signed-off-by: Gengliang Wang --- .../datasources/v2/DataSourceV2Relation.scala | 8 +- .../sql/connector/DataSourceV2Suite.scala | 132 ++++++++++++++++-- 2 files changed, 126 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index b0fb414fce976..4664c5c6a70c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -184,7 +184,13 @@ case class DataSourceV2ScanRelation( relation = this.relation.copy( output = this.relation.output.map(QueryPlan.normalizeExpressions(_, this.relation.output)) ), - output = this.output.map(QueryPlan.normalizeExpressions(_, this.output)) + output = this.output.map(QueryPlan.normalizeExpressions(_, this.output)), + keyGroupedPartitioning = keyGroupedPartitioning.map( + _.map(QueryPlan.normalizeExpressions(_, output)) + ), + ordering = ordering.map( + _.map(o => o.copy(child = QueryPlan.normalizeExpressions(o.child, output))) + ) ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 01fa2b13b86f0..a09b7e0827c49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -21,6 +21,8 @@ import java.io.File import java.util import java.util.OptionalLong +import scala.jdk.CollectionConverters._ + import test.org.apache.spark.sql.connector._ import org.apache.spark.SparkUnsupportedOperationException @@ -37,7 +39,7 @@ import org.apache.spark.sql.connector.read.Scan.ColumnarSupportMode import org.apache.spark.sql.connector.read.partitioning.{KeyGroupedPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation, V2ScanPartitioningAndOrdering} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector @@ -1008,6 +1010,52 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS "Canonicalized DataSourceV2ScanRelation instances should be equal") } + test("SPARK-54163: scan canonicalization for partitioning and ordering aware data source") { + val options = new CaseInsensitiveStringMap(Map( + "partitionKeys" -> "i", + "orderKeys" -> "i,j" + ).asJava) + val table = new OrderAndPartitionAwareDataSource().getTable(options) + + def createDsv2ScanRelation(): DataSourceV2ScanRelation = { + val relation = DataSourceV2Relation.create(table, None, None, options) + val scan = relation.table.asReadable.newScanBuilder(relation.options).build() + val scanRelation = DataSourceV2ScanRelation(relation, scan, relation.output) + // Attach partitioning and ordering information to DataSourceV2ScanRelation + V2ScanPartitioningAndOrdering.apply(scanRelation).asInstanceOf[DataSourceV2ScanRelation] + } + + // Create two DataSourceV2ScanRelation instances, representing the scan of the same table + val scanRelation1 = createDsv2ScanRelation() + val scanRelation2 = createDsv2ScanRelation() + + // assert scanRelations have partitioning and ordering + assert(scanRelation1.keyGroupedPartitioning.isDefined && + scanRelation1.keyGroupedPartitioning.get.nonEmpty, + "DataSourceV2ScanRelation should have key grouped partitioning") + assert(scanRelation1.ordering.isDefined && scanRelation1.ordering.get.nonEmpty, + "DataSourceV2ScanRelation should have ordering") + + // the two instances should not be the same, as they should have different attribute IDs + assert(scanRelation1 != scanRelation2, + "Two created DataSourceV2ScanRelation instances should not be the same") + assert(scanRelation1.output.map(_.exprId).toSet != scanRelation2.output.map(_.exprId).toSet, + "Output attributes should have different expression IDs before canonicalization") + assert(scanRelation1.relation.output.map(_.exprId).toSet != + scanRelation2.relation.output.map(_.exprId).toSet, + "Relation output attributes should have different expression IDs before canonicalization") + assert(scanRelation1.keyGroupedPartitioning.get.flatMap(_.references.map(_.exprId)).toSet != + scanRelation2.keyGroupedPartitioning.get.flatMap(_.references.map(_.exprId)).toSet, + "Partitioning columns should have different expression IDs before canonicalization") + assert(scanRelation1.ordering.get.flatMap(_.references.map(_.exprId)).toSet != + scanRelation2.ordering.get.flatMap(_.references.map(_.exprId)).toSet, + "Ordering columns should have different expression IDs before canonicalization") + + // After canonicalization, the two instances should be equal + assert(scanRelation1.canonicalized == scanRelation2.canonicalized, + "Canonicalized DataSourceV2ScanRelation instances should be equal") + } + test("SPARK-53809: check mergeScalarSubqueries is effective for DataSourceV2ScanRelation") { val df = spark.read.format(classOf[SimpleDataSourceV2].getName).load() df.createOrReplaceTempView("df") @@ -1052,6 +1100,64 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS // Verify the query produces correct results checkAnswer(query, Row(9, 0)) } + + test( + "SPARK-54163: check mergeScalarSubqueries is effective for OrderAndPartitionAwareDataSource" + ) { + withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "true") { + val options = Map( + "partitionKeys" -> "i", + "orderKeys" -> "i,j" + ) + + // Create the OrderAndPartitionAwareDataSource DataFrame + val df = spark.read + .format(classOf[OrderAndPartitionAwareDataSource].getName) + .options(options) + .load() + df.createOrReplaceTempView("df") + + val query = sql("select (select max(i) from df) as max_i, (select min(i) from df) as min_i") + val optimizedPlan = query.queryExecution.optimizedPlan + + // check optimizedPlan merged scalar subqueries `select max(i), min(i) from df` + val sub1 = optimizedPlan.asInstanceOf[Project].projectList.head.collect { + case s: ScalarSubquery => s + } + val sub2 = optimizedPlan.asInstanceOf[Project].projectList(1).collect { + case s: ScalarSubquery => s + } + + // Both subqueries should reference the same merged plan `select max(i), min(i) from df` + assert(sub1.nonEmpty && sub2.nonEmpty, "Both scalar subqueries should exist") + assert(sub1.head.plan == sub2.head.plan, + "Both subqueries should reference the same merged plan") + + // Extract the aggregate from the merged plan sub1 + val agg = sub1.head.plan.collect { + case a: Aggregate => a + }.head + + // Check that the aggregate contains both max(i) and min(i) + val aggFunctionSet = agg.aggregateExpressions.flatMap { expr => + expr.collect { + case ae: org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression => + ae.aggregateFunction + } + }.toSet + + assert(aggFunctionSet.size == 2, "Aggregate should contain exactly two aggregate functions") + assert(aggFunctionSet + .exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.aggregate.Max]), + "Aggregate should contain max(i)") + assert(aggFunctionSet + .exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.aggregate.Min]), + "Aggregate should contain min(i)") + + // Verify the query produces correct results + checkAnswer(query, Row(4, 1)) + } + } } case class RangeInputPartition(start: Int, end: Int) extends InputPartition @@ -1093,6 +1199,18 @@ abstract class SimpleScanBuilder extends ScanBuilder override def readSchema(): StructType = TestingV2Source.schema override def createReaderFactory(): PartitionReaderFactory = SimpleReaderFactory + + override def equals(obj: Any): Boolean = { + obj match { + case s: Scan => + this.readSchema() == s.readSchema() + case _ => false + } + } + + override def hashCode(): Int = { + this.readSchema().hashCode() + } } trait TestingV2Source extends TableProvider { @@ -1157,18 +1275,6 @@ class SimpleDataSourceV2 extends TestingV2Source { override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 5), RangeInputPartition(5, 10)) } - - override def equals(obj: Any): Boolean = { - obj match { - case s: Scan => - this.readSchema() == s.readSchema() - case _ => false - } - } - - override def hashCode(): Int = { - this.readSchema().hashCode() - } } override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { From 6ee5a16f4b4fb36484cefc4841ac003d3fd30666 Mon Sep 17 00:00:00 2001 From: antban Date: Tue, 18 Nov 2025 13:52:13 -0800 Subject: [PATCH 164/400] [SPARK-54349][PYTHON] Refactor code a bit to simplify faulthandler integration extension ### What changes were proposed in this pull request? There are many places where pyspark is trying to integrate with faulthandler and use the same functionality to dump stack traces/record thread dumps. In order to reduce the complexity of the integration and ease the extension of integration it makes sense to technically refactor the code to use the same code in all the places. ### Why are the changes needed? Improves developer experience. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By existing unit tests ### Was this patch authored or co-authored using generative AI tooling? Closes #53016 from antban/simplify-faulthandler-integration. Authored-by: antban Signed-off-by: Takuya Ueshin (cherry picked from commit 6227fbab03082a19ad04ab300b89a721334eb9bb) Signed-off-by: Takuya Ueshin --- python/pyspark/sql/worker/analyze_udtf.py | 27 +++----- .../sql/worker/commit_data_source_write.py | 27 +++----- .../pyspark/sql/worker/create_data_source.py | 27 +++----- .../worker/data_source_pushdown_filters.py | 27 +++----- .../pyspark/sql/worker/lookup_data_sources.py | 27 +++----- .../sql/worker/plan_data_source_read.py | 27 +++----- .../worker/python_streaming_sink_runner.py | 27 +++----- .../sql/worker/write_into_data_source.py | 27 +++----- python/pyspark/util.py | 66 +++++++++++++++++++ python/pyspark/worker.py | 29 +++----- 10 files changed, 138 insertions(+), 173 deletions(-) diff --git a/python/pyspark/sql/worker/analyze_udtf.py b/python/pyspark/sql/worker/analyze_udtf.py index 665b1297fbc1f..526cb316862c7 100644 --- a/python/pyspark/sql/worker/analyze_udtf.py +++ b/python/pyspark/sql/worker/analyze_udtf.py @@ -15,7 +15,6 @@ # limitations under the License. # -import faulthandler import inspect import os import sys @@ -35,7 +34,12 @@ from pyspark.sql.functions import OrderingColumn, PartitioningColumn, SelectedColumn from pyspark.sql.types import _parse_datatype_json_string, StructType from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -100,6 +104,7 @@ def read_arguments(infile: IO) -> Tuple[List[AnalyzeArgument], Dict[str, Analyze return args, kwargs +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Runs the Python UDTF's `analyze` static method. @@ -108,18 +113,10 @@ def main(infile: IO, outfile: IO) -> None: in JVM and receive the Python UDTF and its arguments for the `analyze` static method, and call the `analyze` static method, and send back a AnalyzeResult as a result of the method. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -266,11 +263,6 @@ def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySpar except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -282,9 +274,6 @@ def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySpar write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/commit_data_source_write.py b/python/pyspark/sql/worker/commit_data_source_write.py index fb82b65f31229..37fee6ad8357e 100644 --- a/python/pyspark/sql/worker/commit_data_source_write.py +++ b/python/pyspark/sql/worker/commit_data_source_write.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler import os import sys from typing import IO @@ -29,7 +28,12 @@ SpecialLengths, ) from pyspark.sql.datasource import DataSourceWriter, WriterCommitMessage -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, pickleSer, @@ -40,6 +44,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for committing or aborting a data source write operation. @@ -49,18 +54,10 @@ def main(infile: IO, outfile: IO) -> None: responsible for invoking either the `commit` or the `abort` method on a data source writer instance, given a list of commit messages. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -106,11 +103,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -122,9 +114,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/create_data_source.py b/python/pyspark/sql/worker/create_data_source.py index 15e8fdc618e29..bf6ceda41ffb9 100644 --- a/python/pyspark/sql/worker/create_data_source.py +++ b/python/pyspark/sql/worker/create_data_source.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler import inspect import os import sys @@ -32,7 +31,12 @@ ) from pyspark.sql.datasource import DataSource, CaseInsensitiveDict from pyspark.sql.types import _parse_datatype_json_string, StructType -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -45,6 +49,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for creating a Python data source instance. @@ -62,18 +67,10 @@ def main(infile: IO, outfile: IO) -> None: This process then creates a `DataSource` instance using the above information and sends the pickled instance as well as the schema back to the JVM. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -172,11 +169,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -188,9 +180,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/data_source_pushdown_filters.py b/python/pyspark/sql/worker/data_source_pushdown_filters.py index 8601521bcfb13..7d255e1dbf778 100644 --- a/python/pyspark/sql/worker/data_source_pushdown_filters.py +++ b/python/pyspark/sql/worker/data_source_pushdown_filters.py @@ -16,7 +16,6 @@ # import base64 -import faulthandler import json import os import sys @@ -49,7 +48,12 @@ ) from pyspark.sql.types import StructType, VariantVal, _parse_datatype_json_string from pyspark.sql.worker.plan_data_source_read import write_read_func_and_partitions -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, pickleSer, @@ -119,6 +123,7 @@ def deserializeFilter(jsonDict: dict) -> Filter: return filter +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for planning a data source read with filter pushdown. @@ -140,18 +145,10 @@ def main(infile: IO, outfile: IO) -> None: on the reader and determines which filters are supported. The indices of the supported filters are sent back to the JVM, along with the list of partitions and the read function. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -258,11 +255,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -274,9 +266,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/lookup_data_sources.py b/python/pyspark/sql/worker/lookup_data_sources.py index eeb84263d4452..b23903cac8cb8 100644 --- a/python/pyspark/sql/worker/lookup_data_sources.py +++ b/python/pyspark/sql/worker/lookup_data_sources.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler from importlib import import_module from pkgutil import iter_modules import os @@ -29,7 +28,12 @@ SpecialLengths, ) from pyspark.sql.datasource import DataSource -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, pickleSer, @@ -40,6 +44,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for looking up the available Python Data Sources in Python path. @@ -51,18 +56,10 @@ def main(infile: IO, outfile: IO) -> None: This is responsible for searching the available Python Data Sources so they can be statically registered automatically. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -89,11 +86,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -105,9 +97,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/plan_data_source_read.py b/python/pyspark/sql/worker/plan_data_source_read.py index db79e58f2ec4f..51036f17586f2 100644 --- a/python/pyspark/sql/worker/plan_data_source_read.py +++ b/python/pyspark/sql/worker/plan_data_source_read.py @@ -15,7 +15,6 @@ # limitations under the License. # -import faulthandler import os import sys import functools @@ -47,7 +46,12 @@ BinaryType, StructType, ) -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -267,6 +271,7 @@ def data_source_read_func(iterator: Iterable[pa.RecordBatch]) -> Iterable[pa.Rec write_int(0, outfile) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for planning a data source read. @@ -287,18 +292,10 @@ def main(infile: IO, outfile: IO) -> None: The partition values and the Arrow Batch are then serialized and sent back to the JVM via the socket. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -402,11 +399,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -418,9 +410,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/python_streaming_sink_runner.py b/python/pyspark/sql/worker/python_streaming_sink_runner.py index ed6907ce5b63d..5ca3307fca33c 100644 --- a/python/pyspark/sql/worker/python_streaming_sink_runner.py +++ b/python/pyspark/sql/worker/python_streaming_sink_runner.py @@ -15,7 +15,6 @@ # limitations under the License. # -import faulthandler import os import sys from typing import IO @@ -35,7 +34,12 @@ _parse_datatype_json_string, StructType, ) -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -48,6 +52,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for committing or aborting a data source streaming write operation. @@ -57,18 +62,10 @@ def main(infile: IO, outfile: IO) -> None: responsible for invoking either the `commit` or the `abort` method on a data source writer instance, given a list of commit messages. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() setup_spark_files(infile) setup_broadcasts(infile) @@ -138,11 +135,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -154,9 +146,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index 917d0ca8e0079..b8a54f8397dc8 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler import inspect import os import sys @@ -46,7 +45,12 @@ BinaryType, _create_row, ) -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -59,6 +63,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for saving into a Python data source. @@ -78,18 +83,10 @@ def main(infile: IO, outfile: IO) -> None: instance and send a function using the writer instance that can be used in mapInPandas/mapInArrow back to the JVM. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -264,11 +261,6 @@ def batch_to_rows() -> Iterator[Row]: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -280,9 +272,6 @@ def batch_to_rows() -> Iterator[Row]: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 501d0afbb163f..9935206df1771 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -18,6 +18,7 @@ import copy import functools +import faulthandler import itertools import os import platform @@ -917,6 +918,71 @@ def default_api_mode() -> str: return "classic" +class _FaulthandlerHelper: + def __init__(self) -> None: + self._log_path: Optional[str] = None + self._log_file: Optional[TextIO] = None + self._periodic_traceback = False + + def start(self) -> None: + if self._log_path: + raise Exception("Fault handler is already registered. No second registration allowed") + self._log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + if self._log_path: + self._log_path = os.path.join(self._log_path, str(os.getpid())) + self._log_file = open(self._log_path, "w") + + faulthandler.enable(file=self._log_file) + + def stop(self) -> None: + if self._log_path: + faulthandler.disable() + if self._log_file: + self._log_file.close() + self._log_file = None + try: + os.remove(self._log_path) + finally: + self._log_path = None + if self._periodic_traceback: + faulthandler.cancel_dump_traceback_later() + self._periodic_traceback = False + + def start_periodic_traceback(self) -> None: + # If the registration is already done - do nothing + if self._periodic_traceback: + return + + traceback_dump_interval_seconds = os.environ.get( + "PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None + ) + if traceback_dump_interval_seconds is not None and int(traceback_dump_interval_seconds) > 0: + self._periodic_traceback = True + faulthandler.dump_traceback_later(int(traceback_dump_interval_seconds), repeat=True) + + def with_faulthandler(self, func: Callable) -> Callable: + """ + Registers fault handler for the duration of function execution. + After function execution is over the faulthandler registration is cleaned as well, + including any files created for the integration. + """ + + @functools.wraps(func) + def wrapper(*args: Any, **kwargs: Any) -> Any: + try: + self.start() + return func(*args, **kwargs) + finally: + self.stop() + + return wrapper + + +_faulthandler_helper = _FaulthandlerHelper() +with_faulthandler = _faulthandler_helper.with_faulthandler +start_faulthandler_periodic_traceback = _faulthandler_helper.start_periodic_traceback + + if __name__ == "__main__": if "pypy" not in platform.python_implementation().lower(): import doctest diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 6e34b041665ac..4bae9f6dc48f5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -27,7 +27,6 @@ import itertools import json from typing import Any, Callable, Iterable, Iterator, Optional, Tuple -import faulthandler from pyspark.accumulators import ( SpecialAccumulatorIds, @@ -84,7 +83,12 @@ _create_row, _parse_datatype_json_string, ) -from pyspark.util import fail_on_stopiteration, handle_worker_exception +from pyspark.util import ( + fail_on_stopiteration, + handle_worker_exception, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark import shuffle from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError from pyspark.worker_util import ( @@ -3290,23 +3294,14 @@ def func(_, it): return func, None, ser, ser +@with_faulthandler def main(infile, outfile): - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - boot_time = time.time() split_index = read_int(infile) if split_index == -1: # for unit tests sys.exit(-1) - - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) - + start_faulthandler_periodic_traceback() check_python_version(infile) # read inputs only for a barrier task @@ -3397,11 +3392,6 @@ def process(): except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) write_long(shuffle.MemoryBytesSpilled, outfile) @@ -3419,9 +3409,6 @@ def process(): write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. From 5c47be2a94a9fd3597882f3014bebeeff18127ec Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Tue, 18 Nov 2025 14:56:32 -0800 Subject: [PATCH 165/400] Revert "[SPARK-54349][PYTHON] Refactor code a bit to simplify faulthandler integration extension" This reverts commit 6ee5a16f4b4fb36484cefc4841ac003d3fd30666. --- python/pyspark/sql/worker/analyze_udtf.py | 27 +++++--- .../sql/worker/commit_data_source_write.py | 27 +++++--- .../pyspark/sql/worker/create_data_source.py | 27 +++++--- .../worker/data_source_pushdown_filters.py | 27 +++++--- .../pyspark/sql/worker/lookup_data_sources.py | 27 +++++--- .../sql/worker/plan_data_source_read.py | 27 +++++--- .../worker/python_streaming_sink_runner.py | 27 +++++--- .../sql/worker/write_into_data_source.py | 27 +++++--- python/pyspark/util.py | 66 ------------------- python/pyspark/worker.py | 29 +++++--- 10 files changed, 173 insertions(+), 138 deletions(-) diff --git a/python/pyspark/sql/worker/analyze_udtf.py b/python/pyspark/sql/worker/analyze_udtf.py index 526cb316862c7..665b1297fbc1f 100644 --- a/python/pyspark/sql/worker/analyze_udtf.py +++ b/python/pyspark/sql/worker/analyze_udtf.py @@ -15,6 +15,7 @@ # limitations under the License. # +import faulthandler import inspect import os import sys @@ -34,12 +35,7 @@ from pyspark.sql.functions import OrderingColumn, PartitioningColumn, SelectedColumn from pyspark.sql.types import _parse_datatype_json_string, StructType from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, read_command, @@ -104,7 +100,6 @@ def read_arguments(infile: IO) -> Tuple[List[AnalyzeArgument], Dict[str, Analyze return args, kwargs -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Runs the Python UDTF's `analyze` static method. @@ -113,10 +108,18 @@ def main(infile: IO, outfile: IO) -> None: in JVM and receive the Python UDTF and its arguments for the `analyze` static method, and call the `analyze` static method, and send back a AnalyzeResult as a result of the method. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -263,6 +266,11 @@ def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySpar except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -274,6 +282,9 @@ def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySpar write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/commit_data_source_write.py b/python/pyspark/sql/worker/commit_data_source_write.py index 37fee6ad8357e..fb82b65f31229 100644 --- a/python/pyspark/sql/worker/commit_data_source_write.py +++ b/python/pyspark/sql/worker/commit_data_source_write.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import faulthandler import os import sys from typing import IO @@ -28,12 +29,7 @@ SpecialLengths, ) from pyspark.sql.datasource import DataSourceWriter, WriterCommitMessage -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, pickleSer, @@ -44,7 +40,6 @@ ) -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for committing or aborting a data source write operation. @@ -54,10 +49,18 @@ def main(infile: IO, outfile: IO) -> None: responsible for invoking either the `commit` or the `abort` method on a data source writer instance, given a list of commit messages. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -103,6 +106,11 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -114,6 +122,9 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/create_data_source.py b/python/pyspark/sql/worker/create_data_source.py index bf6ceda41ffb9..15e8fdc618e29 100644 --- a/python/pyspark/sql/worker/create_data_source.py +++ b/python/pyspark/sql/worker/create_data_source.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import faulthandler import inspect import os import sys @@ -31,12 +32,7 @@ ) from pyspark.sql.datasource import DataSource, CaseInsensitiveDict from pyspark.sql.types import _parse_datatype_json_string, StructType -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, read_command, @@ -49,7 +45,6 @@ ) -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for creating a Python data source instance. @@ -67,10 +62,18 @@ def main(infile: IO, outfile: IO) -> None: This process then creates a `DataSource` instance using the above information and sends the pickled instance as well as the schema back to the JVM. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -169,6 +172,11 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -180,6 +188,9 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/data_source_pushdown_filters.py b/python/pyspark/sql/worker/data_source_pushdown_filters.py index 7d255e1dbf778..8601521bcfb13 100644 --- a/python/pyspark/sql/worker/data_source_pushdown_filters.py +++ b/python/pyspark/sql/worker/data_source_pushdown_filters.py @@ -16,6 +16,7 @@ # import base64 +import faulthandler import json import os import sys @@ -48,12 +49,7 @@ ) from pyspark.sql.types import StructType, VariantVal, _parse_datatype_json_string from pyspark.sql.worker.plan_data_source_read import write_read_func_and_partitions -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, pickleSer, @@ -123,7 +119,6 @@ def deserializeFilter(jsonDict: dict) -> Filter: return filter -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for planning a data source read with filter pushdown. @@ -145,10 +140,18 @@ def main(infile: IO, outfile: IO) -> None: on the reader and determines which filters are supported. The indices of the supported filters are sent back to the JVM, along with the list of partitions and the read function. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -255,6 +258,11 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -266,6 +274,9 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/lookup_data_sources.py b/python/pyspark/sql/worker/lookup_data_sources.py index b23903cac8cb8..eeb84263d4452 100644 --- a/python/pyspark/sql/worker/lookup_data_sources.py +++ b/python/pyspark/sql/worker/lookup_data_sources.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import faulthandler from importlib import import_module from pkgutil import iter_modules import os @@ -28,12 +29,7 @@ SpecialLengths, ) from pyspark.sql.datasource import DataSource -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, pickleSer, @@ -44,7 +40,6 @@ ) -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for looking up the available Python Data Sources in Python path. @@ -56,10 +51,18 @@ def main(infile: IO, outfile: IO) -> None: This is responsible for searching the available Python Data Sources so they can be statically registered automatically. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -86,6 +89,11 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -97,6 +105,9 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/plan_data_source_read.py b/python/pyspark/sql/worker/plan_data_source_read.py index 51036f17586f2..db79e58f2ec4f 100644 --- a/python/pyspark/sql/worker/plan_data_source_read.py +++ b/python/pyspark/sql/worker/plan_data_source_read.py @@ -15,6 +15,7 @@ # limitations under the License. # +import faulthandler import os import sys import functools @@ -46,12 +47,7 @@ BinaryType, StructType, ) -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, read_command, @@ -271,7 +267,6 @@ def data_source_read_func(iterator: Iterable[pa.RecordBatch]) -> Iterable[pa.Rec write_int(0, outfile) -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for planning a data source read. @@ -292,10 +287,18 @@ def main(infile: IO, outfile: IO) -> None: The partition values and the Arrow Batch are then serialized and sent back to the JVM via the socket. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -399,6 +402,11 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -410,6 +418,9 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/python_streaming_sink_runner.py b/python/pyspark/sql/worker/python_streaming_sink_runner.py index 5ca3307fca33c..ed6907ce5b63d 100644 --- a/python/pyspark/sql/worker/python_streaming_sink_runner.py +++ b/python/pyspark/sql/worker/python_streaming_sink_runner.py @@ -15,6 +15,7 @@ # limitations under the License. # +import faulthandler import os import sys from typing import IO @@ -34,12 +35,7 @@ _parse_datatype_json_string, StructType, ) -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, read_command, @@ -52,7 +48,6 @@ ) -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for committing or aborting a data source streaming write operation. @@ -62,10 +57,18 @@ def main(infile: IO, outfile: IO) -> None: responsible for invoking either the `commit` or the `abort` method on a data source writer instance, given a list of commit messages. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) setup_spark_files(infile) setup_broadcasts(infile) @@ -135,6 +138,11 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -146,6 +154,9 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index b8a54f8397dc8..917d0ca8e0079 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import faulthandler import inspect import os import sys @@ -45,12 +46,7 @@ BinaryType, _create_row, ) -from pyspark.util import ( - handle_worker_exception, - local_connect_and_auth, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import handle_worker_exception, local_connect_and_auth from pyspark.worker_util import ( check_python_version, read_command, @@ -63,7 +59,6 @@ ) -@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for saving into a Python data source. @@ -83,10 +78,18 @@ def main(infile: IO, outfile: IO) -> None: instance and send a function using the writer instance that can be used in mapInPandas/mapInArrow back to the JVM. """ + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + check_python_version(infile) - start_faulthandler_periodic_traceback() + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -261,6 +264,11 @@ def batch_to_rows() -> Iterator[Row]: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -272,6 +280,9 @@ def batch_to_rows() -> Iterator[Row]: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 9935206df1771..501d0afbb163f 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -18,7 +18,6 @@ import copy import functools -import faulthandler import itertools import os import platform @@ -918,71 +917,6 @@ def default_api_mode() -> str: return "classic" -class _FaulthandlerHelper: - def __init__(self) -> None: - self._log_path: Optional[str] = None - self._log_file: Optional[TextIO] = None - self._periodic_traceback = False - - def start(self) -> None: - if self._log_path: - raise Exception("Fault handler is already registered. No second registration allowed") - self._log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - if self._log_path: - self._log_path = os.path.join(self._log_path, str(os.getpid())) - self._log_file = open(self._log_path, "w") - - faulthandler.enable(file=self._log_file) - - def stop(self) -> None: - if self._log_path: - faulthandler.disable() - if self._log_file: - self._log_file.close() - self._log_file = None - try: - os.remove(self._log_path) - finally: - self._log_path = None - if self._periodic_traceback: - faulthandler.cancel_dump_traceback_later() - self._periodic_traceback = False - - def start_periodic_traceback(self) -> None: - # If the registration is already done - do nothing - if self._periodic_traceback: - return - - traceback_dump_interval_seconds = os.environ.get( - "PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None - ) - if traceback_dump_interval_seconds is not None and int(traceback_dump_interval_seconds) > 0: - self._periodic_traceback = True - faulthandler.dump_traceback_later(int(traceback_dump_interval_seconds), repeat=True) - - def with_faulthandler(self, func: Callable) -> Callable: - """ - Registers fault handler for the duration of function execution. - After function execution is over the faulthandler registration is cleaned as well, - including any files created for the integration. - """ - - @functools.wraps(func) - def wrapper(*args: Any, **kwargs: Any) -> Any: - try: - self.start() - return func(*args, **kwargs) - finally: - self.stop() - - return wrapper - - -_faulthandler_helper = _FaulthandlerHelper() -with_faulthandler = _faulthandler_helper.with_faulthandler -start_faulthandler_periodic_traceback = _faulthandler_helper.start_periodic_traceback - - if __name__ == "__main__": if "pypy" not in platform.python_implementation().lower(): import doctest diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4bae9f6dc48f5..6e34b041665ac 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -27,6 +27,7 @@ import itertools import json from typing import Any, Callable, Iterable, Iterator, Optional, Tuple +import faulthandler from pyspark.accumulators import ( SpecialAccumulatorIds, @@ -83,12 +84,7 @@ _create_row, _parse_datatype_json_string, ) -from pyspark.util import ( - fail_on_stopiteration, - handle_worker_exception, - with_faulthandler, - start_faulthandler_periodic_traceback, -) +from pyspark.util import fail_on_stopiteration, handle_worker_exception from pyspark import shuffle from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError from pyspark.worker_util import ( @@ -3294,14 +3290,23 @@ def func(_, it): return func, None, ser, ser -@with_faulthandler def main(infile, outfile): + faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: + if faulthandler_log_path: + faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) + faulthandler_log_file = open(faulthandler_log_path, "w") + faulthandler.enable(file=faulthandler_log_file) + boot_time = time.time() split_index = read_int(infile) if split_index == -1: # for unit tests sys.exit(-1) - start_faulthandler_periodic_traceback() + + if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: + faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + check_python_version(infile) # read inputs only for a barrier task @@ -3392,6 +3397,11 @@ def process(): except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) + finally: + if faulthandler_log_path: + faulthandler.disable() + faulthandler_log_file.close() + os.remove(faulthandler_log_path) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) write_long(shuffle.MemoryBytesSpilled, outfile) @@ -3409,6 +3419,9 @@ def process(): write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) + # Force to cancel dump_traceback_later + faulthandler.cancel_dump_traceback_later() + if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. From 8e769b61e0a0a337ec2c4153a59a59ebd17f85aa Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 19 Nov 2025 12:28:14 +0800 Subject: [PATCH 166/400] [SPARK-54375][CONNECT][TESTS][FOLLOWUP] Make `PythonPipelineSuite` perform a default check for `PythonTestDepsChecker.isConnectDepsAvailable` ### What changes were proposed in this pull request? This pr aims to make the test cases in `PythonPipelineSuite` perform a default check for `PythonTestDepsChecker.isConnectDepsAvailable`. ### Why are the changes needed? Simplify the dependency checks for Python modules in test cases within `PythonPipelineSuite`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #53106 from LuciferYang/refactor-PythonPipelineSuite. Authored-by: yangjie01 Signed-off-by: yangjie01 (cherry picked from commit 1aa8d5aaa11d062f57aa0184e1b75d23cf974930) Signed-off-by: yangjie01 --- .../pipelines/PythonPipelineSuite.scala | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala index 826e2338589da..98b33c3296fac 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala @@ -26,6 +26,9 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable.ArrayBuffer import scala.util.Try +import org.scalactic.source.Position +import org.scalatest.Tag + import org.apache.spark.api.python.PythonUtils import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier @@ -49,7 +52,6 @@ class PythonPipelineSuite with EventVerificationTestHelpers { def buildGraph(pythonText: String): DataflowGraph = { - assume(PythonTestDepsChecker.isConnectDepsAvailable) val indentedPythonText = pythonText.linesIterator.map(" " + _).mkString("\n") // create a unique identifier to allow identifying the session and dataflow graph val customSessionIdentifier = UUID.randomUUID().toString @@ -530,7 +532,6 @@ class PythonPipelineSuite "eager analysis or execution will fail")( Seq("""spark.sql("SELECT * FROM src")""", """spark.read.table("src").collect()""")) { command => - assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |@dp.materialized_view @@ -549,7 +550,6 @@ class PythonPipelineSuite } test("create dataset with the same name will fail") { - assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[AnalysisException] { buildGraph(s""" |@dp.materialized_view @@ -623,7 +623,6 @@ class PythonPipelineSuite } test("create datasets with three part names") { - assume(PythonTestDepsChecker.isConnectDepsAvailable) val graphTry = Try { buildGraph(s""" |@dp.table(name = "some_catalog.some_schema.mv") @@ -676,7 +675,6 @@ class PythonPipelineSuite } test("create named flow with multipart name will fail") { - assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |@dp.table @@ -825,7 +823,6 @@ class PythonPipelineSuite } test("create pipeline without table will throw RUN_EMPTY_PIPELINE exception") { - assume(PythonTestDepsChecker.isConnectDepsAvailable) checkError( exception = intercept[AnalysisException] { buildGraph(s""" @@ -837,7 +834,6 @@ class PythonPipelineSuite } test("create pipeline with only temp view will throw RUN_EMPTY_PIPELINE exception") { - assume(PythonTestDepsChecker.isConnectDepsAvailable) checkError( exception = intercept[AnalysisException] { buildGraph(s""" @@ -851,7 +847,6 @@ class PythonPipelineSuite } test("create pipeline with only flow will throw RUN_EMPTY_PIPELINE exception") { - assume(PythonTestDepsChecker.isConnectDepsAvailable) checkError( exception = intercept[AnalysisException] { buildGraph(s""" @@ -1048,7 +1043,6 @@ class PythonPipelineSuite gridTest("Unsupported SQL command outside query function should result in a failure")( unsupportedSqlCommandList) { unsupportedSqlCommand => - assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |spark.sql("$unsupportedSqlCommand") @@ -1063,7 +1057,6 @@ class PythonPipelineSuite gridTest("Unsupported SQL command inside query function should result in a failure")( unsupportedSqlCommandList) { unsupportedSqlCommand => - assume(PythonTestDepsChecker.isConnectDepsAvailable) val ex = intercept[RuntimeException] { buildGraph(s""" |@dp.materialized_view() @@ -1111,4 +1104,13 @@ class PythonPipelineSuite | return spark.range(5) |""".stripMargin) } + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + if (PythonTestDepsChecker.isConnectDepsAvailable) { + super.test(testName, testTags: _*)(testFun) + } else { + super.ignore(testName, testTags: _*)(testFun) + } + } } From 2daaa090a09565aa1c6927bf8f13fade98bd2186 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Wed, 19 Nov 2025 20:09:14 +0800 Subject: [PATCH 167/400] [SPARK-54253][GEO][SQL] Add a guarding config for geospatial support ### What changes were proposed in this pull request? Introduce a new SQL config for controlling the geospatial feature: ``` spark.sql.geospatial.enabled ``` The default value is `false`, and enabled only in testing. ### Why are the changes needed? Guard the geospatial feature until it's fully finished. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added appropriate unit tests to confirm that the config is effective: - `STExpressionsSuite` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53009 from uros-db/geo-config. Authored-by: Uros Bojanic Signed-off-by: Wenchen Fan (cherry picked from commit d29968498a5dfc660d4921bef1410d36cf7756d5) Signed-off-by: Wenchen Fan --- .../spark/unsafe/types/GeographyVal.java | 3 + .../spark/unsafe/types/GeometryVal.java | 3 + .../resources/error/error-conditions.json | 5 ++ .../org/apache/spark/sql/types/Geography.java | 3 + .../org/apache/spark/sql/types/Geometry.java | 3 + ...CartesianSpatialReferenceSystemMapper.java | 3 + ...eographicSpatialReferenceSystemMapper.java | 3 + .../types/SpatialReferenceSystemCache.java | 3 + .../SpatialReferenceSystemInformation.java | 3 + .../types/SpatialReferenceSystemMapper.java | 4 ++ .../spark/sql/types/GeographyType.scala | 10 +-- .../apache/spark/sql/types/GeometryType.scala | 8 +-- .../apache/spark/sql/types/SpatialType.scala | 3 +- .../sql/catalyst/CatalystTypeConverters.scala | 38 +++++++++-- .../catalyst/DeserializerBuildHelper.scala | 5 ++ .../sql/catalyst/SerializerBuildHelper.scala | 4 ++ .../spark/sql/catalyst/expressions/Cast.scala | 12 ++++ .../expressions/st/STExpressionUtils.scala | 7 ++ .../expressions/st/stExpressions.scala | 28 ++++++-- .../apache/spark/sql/internal/SQLConf.scala | 9 +++ .../spark/sql/GeographyDataFrameSuite.scala | 32 +++++++++ .../spark/sql/GeometryDataFrameSuite.scala | 32 +++++++++ .../apache/spark/sql/STExpressionsSuite.scala | 68 +++++++++++++++++++ .../apache/spark/sql/STFunctionsSuite.scala | 22 ++++++ 24 files changed, 291 insertions(+), 20 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeographyVal.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeographyVal.java index 48dc6f896e91a..48b121ba894a5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeographyVal.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeographyVal.java @@ -17,9 +17,12 @@ package org.apache.spark.unsafe.types; +import org.apache.spark.annotation.Unstable; + import java.io.Serializable; // This class represents the physical type for the GEOGRAPHY data type. +@Unstable public final class GeographyVal implements Comparable, Serializable { // The GEOGRAPHY type is implemented as a byte array. We provide `getBytes` and `fromBytes` diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeometryVal.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeometryVal.java index 2bb7f194c940d..381d3e25c68af 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeometryVal.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/GeometryVal.java @@ -17,9 +17,12 @@ package org.apache.spark.unsafe.types; +import org.apache.spark.annotation.Unstable; + import java.io.Serializable; // This class represents the physical type for the GEOMETRY data type. +@Unstable public final class GeometryVal implements Comparable, Serializable { // The GEOMETRY type is implemented as a byte array. We provide `getBytes` and `fromBytes` diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 308f49b097fc4..b190e36a5029b 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -6554,6 +6554,11 @@ "Drop the namespace ." ] }, + "GEOSPATIAL_DISABLED" : { + "message" : [ + "Geospatial feature is disabled." + ] + }, "HIVE_TABLE_TYPE" : { "message" : [ "The is hive ." diff --git a/sql/api/src/main/java/org/apache/spark/sql/types/Geography.java b/sql/api/src/main/java/org/apache/spark/sql/types/Geography.java index e99902336ffe9..4a52288ba3f2a 100644 --- a/sql/api/src/main/java/org/apache/spark/sql/types/Geography.java +++ b/sql/api/src/main/java/org/apache/spark/sql/types/Geography.java @@ -17,10 +17,13 @@ package org.apache.spark.sql.types; +import org.apache.spark.annotation.Unstable; + import java.io.Serializable; import java.util.Arrays; // This class represents the Geography data for clients. +@Unstable public final class Geography implements Serializable { // The GEOGRAPHY type is implemented as WKB bytes + SRID integer stored in class itself. protected final byte[] value; diff --git a/sql/api/src/main/java/org/apache/spark/sql/types/Geometry.java b/sql/api/src/main/java/org/apache/spark/sql/types/Geometry.java index 3974aec131d00..fdd64f482130e 100644 --- a/sql/api/src/main/java/org/apache/spark/sql/types/Geometry.java +++ b/sql/api/src/main/java/org/apache/spark/sql/types/Geometry.java @@ -17,10 +17,13 @@ package org.apache.spark.sql.types; +import org.apache.spark.annotation.Unstable; + import java.io.Serializable; import java.util.Arrays; // This class represents the Geometry data for clients. +@Unstable public final class Geometry implements Serializable { // The GEOMETRY type is implemented as WKB bytes + SRID integer stored in class itself. protected final byte[] value; diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/CartesianSpatialReferenceSystemMapper.java b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/CartesianSpatialReferenceSystemMapper.java index d7729e88a3331..7384bb331d44f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/CartesianSpatialReferenceSystemMapper.java +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/CartesianSpatialReferenceSystemMapper.java @@ -17,9 +17,12 @@ package org.apache.spark.sql.internal.types; +import org.apache.spark.annotation.Unstable; + /** * Class for providing SRS mappings for cartesian spatial reference systems. */ +@Unstable public class CartesianSpatialReferenceSystemMapper extends SpatialReferenceSystemMapper { // Returns the string ID corresponding to the input SRID. If not supported, returns `null`. public static String getStringId(int srid) { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/GeographicSpatialReferenceSystemMapper.java b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/GeographicSpatialReferenceSystemMapper.java index 85e0cc53658c6..f409041c564ee 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/GeographicSpatialReferenceSystemMapper.java +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/GeographicSpatialReferenceSystemMapper.java @@ -17,9 +17,12 @@ package org.apache.spark.sql.internal.types; +import org.apache.spark.annotation.Unstable; + /** * Class for providing SRS mappings for geographic spatial reference systems. */ +@Unstable public class GeographicSpatialReferenceSystemMapper extends SpatialReferenceSystemMapper { // Returns the string ID corresponding to the input SRID. If not supported, returns `null`. public static String getStringId(int srid) { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemCache.java b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemCache.java index 40094785600fb..34ea42271352f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemCache.java +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemCache.java @@ -17,12 +17,15 @@ package org.apache.spark.sql.internal.types; +import org.apache.spark.annotation.Unstable; + import java.util.HashMap; import java.util.List; /** * Class for maintaining the mappings between supported SRID/CRS values and the corresponding SRS. */ +@Unstable public class SpatialReferenceSystemCache { // Private constructor to prevent external instantiation of this singleton class. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemInformation.java b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemInformation.java index 46fbfd9e9ac17..ba3526d84f079 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemInformation.java +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemInformation.java @@ -17,9 +17,12 @@ package org.apache.spark.sql.internal.types; +import org.apache.spark.annotation.Unstable; + /** * Class for maintaining information about a spatial reference system (SRS). */ +@Unstable public record SpatialReferenceSystemInformation( // Field storing the spatial reference identifier (SRID) value of this SRS. int srid, diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemMapper.java b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemMapper.java index 2993ba05b76f8..24b82f540b082 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemMapper.java +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/SpatialReferenceSystemMapper.java @@ -17,9 +17,13 @@ package org.apache.spark.sql.internal.types; +import org.apache.spark.annotation.Unstable; + /** * Abstract class for providing SRS mappings for spatial reference systems. */ + +@Unstable public abstract class SpatialReferenceSystemMapper { protected static final SpatialReferenceSystemCache srsCache = SpatialReferenceSystemCache.getInstance(); diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala index 4d6ab7e9c8e50..f74cddc02ac4a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/GeographyType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.{JString, JValue} import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Unstable import org.apache.spark.sql.internal.types.GeographicSpatialReferenceSystemMapper /** @@ -28,7 +28,7 @@ import org.apache.spark.sql.internal.types.GeographicSpatialReferenceSystemMappe * Geospatial Consortium (OGC) Simple Feature Access specification * (https://portal.ogc.org/files/?artifact_id=25355), with a geographic coordinate system. */ -@Experimental +@Unstable class GeographyType private (val crs: String, val algorithm: EdgeInterpolationAlgorithm) extends AtomicType with Serializable { @@ -156,7 +156,7 @@ class GeographyType private (val crs: String, val algorithm: EdgeInterpolationAl } } -@Experimental +@Unstable object GeographyType extends SpatialType { /** @@ -175,7 +175,7 @@ object GeographyType extends SpatialType { /** * The default concrete GeographyType in SQL. */ - private final val GEOGRAPHY_MIXED_TYPE: GeographyType = + private final lazy val GEOGRAPHY_MIXED_TYPE: GeographyType = GeographyType(MIXED_CRS, GEOGRAPHY_DEFAULT_ALGORITHM) /** Returns whether the given SRID is supported. */ @@ -254,8 +254,10 @@ object GeographyType extends SpatialType { * Edge interpolation algorithm for Geography logical type. Currently, Spark only supports * spherical algorithm. */ +@Unstable sealed abstract class EdgeInterpolationAlgorithm +@Unstable object EdgeInterpolationAlgorithm { case object SPHERICAL extends EdgeInterpolationAlgorithm diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala index ff4a720118407..c8b475dae2bab 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/GeometryType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.{JString, JValue} import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Unstable import org.apache.spark.sql.internal.types.CartesianSpatialReferenceSystemMapper /** @@ -28,7 +28,7 @@ import org.apache.spark.sql.internal.types.CartesianSpatialReferenceSystemMapper * Geospatial Consortium (OGC) Simple Feature Access specification * (https://portal.ogc.org/files/?artifact_id=25355), with a Cartesian coordinate system. */ -@Experimental +@Unstable class GeometryType private (val crs: String) extends AtomicType with Serializable { /** @@ -153,7 +153,7 @@ class GeometryType private (val crs: String) extends AtomicType with Serializabl } } -@Experimental +@Unstable object GeometryType extends SpatialType { /** @@ -167,7 +167,7 @@ object GeometryType extends SpatialType { /** * The default concrete GeometryType in SQL. */ - private final val GEOMETRY_MIXED_TYPE: GeometryType = + private final lazy val GEOMETRY_MIXED_TYPE: GeometryType = GeometryType(MIXED_CRS) /** Returns whether the given SRID is supported. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/SpatialType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/SpatialType.scala index a61158e36c850..92fa6dfd38fb1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/SpatialType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/SpatialType.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.types -import org.apache.spark.sql.types.AbstractDataType +import org.apache.spark.annotation.Unstable +@Unstable trait SpatialType extends AbstractDataType { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index b8eee5e1c7c6e..53803192cb3d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -69,6 +69,10 @@ object CatalystTypeConverters { case CharType(length) => new CharConverter(length) case VarcharType(length) => new VarcharConverter(length) case _: StringType => StringConverter + case _ @ (_: GeographyType | _: GeometryType) if !SQLConf.get.geospatialEnabled => + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) case g: GeographyType => new GeographyConverter(g) case g: GeometryType => @@ -349,10 +353,19 @@ object CatalystTypeConverters { row.getUTF8String(column).toString } + private def assertGeospatialEnabled(): Unit = { + if (!SQLConf.get.geospatialEnabled) { + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) + } + } + private class GeometryConverter(dataType: GeometryType) extends CatalystTypeConverter[Any, org.apache.spark.sql.types.Geometry, GeometryVal] { override def toCatalystImpl(scalaValue: Any): GeometryVal = scalaValue match { - case g: org.apache.spark.sql.types.Geometry => STUtils.serializeGeomFromWKB(g, dataType) + case g: org.apache.spark.sql.types.Geometry if SQLConf.get.geospatialEnabled => + STUtils.serializeGeomFromWKB(g, dataType) case other => throw new SparkIllegalArgumentException( errorClass = "_LEGACY_ERROR_TEMP_3219", messageParameters = scala.collection.immutable.Map( @@ -360,17 +373,24 @@ object CatalystTypeConverters { "otherClass" -> other.getClass.getCanonicalName, "dataType" -> StringType.sql)) } - override def toScala(catalystValue: GeometryVal): org.apache.spark.sql.types.Geometry = + override def toScala(catalystValue: GeometryVal): org.apache.spark.sql.types.Geometry = { + assertGeospatialEnabled() if (catalystValue == null) null else STUtils.deserializeGeom(catalystValue, dataType) - override def toScalaImpl(row: InternalRow, column: Int): org.apache.spark.sql.types.Geometry = + } + + override def toScalaImpl(row: InternalRow, column: Int): + org.apache.spark.sql.types.Geometry = { + assertGeospatialEnabled() STUtils.deserializeGeom(row.getGeometry(0), dataType) + } } private class GeographyConverter(dataType: GeographyType) extends CatalystTypeConverter[Any, org.apache.spark.sql.types.Geography, GeographyVal] { override def toCatalystImpl(scalaValue: Any): GeographyVal = scalaValue match { - case g: org.apache.spark.sql.types.Geography => STUtils.serializeGeogFromWKB(g, dataType) + case g: org.apache.spark.sql.types.Geography if SQLConf.get.geospatialEnabled => + STUtils.serializeGeogFromWKB(g, dataType) case other => throw new SparkIllegalArgumentException( errorClass = "_LEGACY_ERROR_TEMP_3219", messageParameters = scala.collection.immutable.Map( @@ -378,11 +398,17 @@ object CatalystTypeConverters { "otherClass" -> other.getClass.getCanonicalName, "dataType" -> StringType.sql)) } - override def toScala(catalystValue: GeographyVal): org.apache.spark.sql.types.Geography = + override def toScala(catalystValue: GeographyVal): org.apache.spark.sql.types.Geography = { + assertGeospatialEnabled() if (catalystValue == null) null else STUtils.deserializeGeog(catalystValue, dataType) - override def toScalaImpl(row: InternalRow, column: Int): org.apache.spark.sql.types.Geography = + } + + override def toScalaImpl(row: InternalRow, column: Int): + org.apache.spark.sql.types.Geography = { + assertGeospatialEnabled() STUtils.deserializeGeog(row.getGeography(0), dataType) + } } private object DateConverter extends CatalystTypeConverter[Any, Date, Any] { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index 60de179edb799..85a5cf4f6b26a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{dataTypeForClass, ex import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, IsNull, Literal, MapKeys, MapValues, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, DecodeUsingSerializer, InitializeJavaBean, Invoke, NewInstance, StaticInvoke, UnresolvedCatalystToExternalMap, UnresolvedMapObjects, WrapOption} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CharVarcharCodegenUtils, DateTimeUtils, IntervalUtils, STUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ object DeserializerBuildHelper { @@ -308,6 +309,10 @@ object DeserializerBuildHelper { "withName", createDeserializerForString(path, returnNullable = false) :: Nil, returnNullable = false) + case _ @ (_: GeographyEncoder | _: GeometryEncoder) if !SQLConf.get.geospatialEnabled => + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) case g: GeographyEncoder => createDeserializerForGeographyType(path, g.dt) case g: GeometryEncoder => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index 06267bca02189..32fb859745d89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -344,6 +344,10 @@ object SerializerBuildHelper { case BoxedDoubleEncoder => createSerializerForDouble(input) case JavaEnumEncoder(_) => createSerializerForJavaEnum(input) case ScalaEnumEncoder(_, _) => createSerializerForScalaEnum(input) + case _ @ (_: GeographyEncoder | _: GeometryEncoder) if !SQLConf.get.geospatialEnabled => + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) case g: GeographyEncoder => createSerializerForGeographyType(input, g.dt) case g: GeometryEncoder => createSerializerForGeometryType(input, g.dt) case CharEncoder(length) => createSerializerForChar(input, length) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 10f4c5c00f043..1f2805ec27899 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -90,6 +90,12 @@ object Cast extends QueryErrorsBase { * - String <=> Binary */ def canAnsiCast(from: DataType, to: DataType): Boolean = (from, to) match { + case (fromType, toType) if !SQLConf.get.geospatialEnabled && + (isGeoSpatialType(fromType) || isGeoSpatialType(toType)) => + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) + case (fromType, toType) if fromType == toType => true case (NullType, _) => true @@ -218,6 +224,12 @@ object Cast extends QueryErrorsBase { * Returns true iff we can cast `from` type to `to` type. */ def canCast(from: DataType, to: DataType): Boolean = (from, to) match { + case (fromType, toType) if !SQLConf.get.geospatialEnabled && + (isGeoSpatialType(fromType) || isGeoSpatialType(toType)) => + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) + case (fromType, toType) if fromType == toType => true case (NullType, _) => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala index 7b0345d277bc1..bfecf8c28ef45 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/STExpressionUtils.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.catalyst.expressions.st +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ private[sql] object STExpressionUtils { @@ -36,6 +38,11 @@ private[sql] object STExpressionUtils { */ def geospatialTypeWithSrid(sourceType: DataType, srid: Expression): DataType = { sourceType match { + case _ if !SQLConf.get.geospatialEnabled => + throw new AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = Map.empty + ) case _: GeometryType => geometryTypeWithSrid(srid) case _: GeographyType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala index 4cdfbeead8753..0a032d191a26f 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/st/stExpressions.scala @@ -17,12 +17,30 @@ package org.apache.spark.sql.catalyst.expressions.st +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.util.{Geography, Geometry, STUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +/** + * ST expressions are behind a feature flag while the geospatial module is under development. + */ + +sealed trait GeospatialInputTypes extends ImplicitCastInputTypes { + override def checkInputDataTypes(): TypeCheckResult = { + if (!SQLConf.get.geospatialEnabled) { + throw new AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = Map.empty + ) + } + super.checkInputDataTypes() + } +} //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines expressions for geospatial operations. @@ -61,7 +79,7 @@ private[sql] object ExpressionDefaults { ) case class ST_AsBinary(geo: Expression) extends RuntimeReplaceable - with ImplicitCastInputTypes + with GeospatialInputTypes with UnaryLike[Expression] { override def inputTypes: Seq[AbstractDataType] = Seq( @@ -109,7 +127,7 @@ case class ST_AsBinary(geo: Expression) ) case class ST_GeogFromWKB(wkb: Expression) extends RuntimeReplaceable - with ImplicitCastInputTypes + with GeospatialInputTypes with UnaryLike[Expression] { override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) @@ -153,7 +171,7 @@ case class ST_GeogFromWKB(wkb: Expression) ) case class ST_GeomFromWKB(wkb: Expression) extends RuntimeReplaceable - with ImplicitCastInputTypes + with GeospatialInputTypes with UnaryLike[Expression] { override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) @@ -201,7 +219,7 @@ case class ST_GeomFromWKB(wkb: Expression) ) case class ST_Srid(geo: Expression) extends RuntimeReplaceable - with ImplicitCastInputTypes + with GeospatialInputTypes with UnaryLike[Expression] { override def inputTypes: Seq[AbstractDataType] = Seq( @@ -249,7 +267,7 @@ case class ST_Srid(geo: Expression) ) case class ST_SetSrid(geo: Expression, srid: Expression) extends RuntimeReplaceable - with ImplicitCastInputTypes + with GeospatialInputTypes with BinaryLike[Expression] { override def inputTypes: Seq[AbstractDataType] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f6aae1d2ba052..940460620da09 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -570,6 +570,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val GEOSPATIAL_ENABLED = + buildConf("spark.sql.geospatial.enabled") + .doc("When true, enables geospatial types (GEOGRAPHY/GEOMETRY) and ST functions.") + .version("4.1.0") + .booleanConf + .createWithDefaultFunction(() => Utils.isTesting) + val EXTENDED_EXPLAIN_PROVIDERS = buildConf("spark.sql.extendedExplainProviders") .doc("A comma-separated list of classes that implement the" + " org.apache.spark.sql.ExtendedExplainGenerator trait. If provided, Spark will print" + @@ -6778,6 +6785,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def nameResolutionLogLevel: Level = getConf(NAME_RESOLUTION_LOG_LEVEL) + def geospatialEnabled: Boolean = getConf(GEOSPATIAL_ENABLED) + def dataSourceV2JoinPushdown: Boolean = getConf(DATA_SOURCE_V2_JOIN_PUSHDOWN) def dynamicPartitionPruningEnabled: Boolean = getConf(DYNAMIC_PARTITION_PRUNING_ENABLED) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala index eeb1ba5ea9e25..537c943e94ae4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeographyDataFrameSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.collection.immutable.Seq import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -177,4 +178,35 @@ class GeographyDataFrameSuite extends QueryTest with SharedSparkSession { val expectedGeog = Geography.fromWKB(pointBytes, 4326) checkAnswer(df, Seq(Row(expectedGeog))) } + + test("geospatial feature disabled") { + withSQLConf(SQLConf.GEOSPATIAL_ENABLED.key -> "false") { + val geography = Geography.fromWKB(point1, 4326) + val schema = StructType(Seq(StructField("col1", GeographyType(4326)))) + // RDD[Row] + schema. + val rdd = sparkContext.parallelize(Seq(Row(geography))) + checkError( + exception = intercept[AnalysisException] { + spark.createDataFrame(rdd, schema).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + // Java List[Row] + schema. + val javaList = java.util.Arrays.asList(Row(geography)) + checkError( + exception = intercept[AnalysisException] { + spark.createDataFrame(javaList, schema).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + // Implicit encoder path. + import testImplicits._ + checkError( + exception = intercept[AnalysisException] { + Seq(geography).toDF("g").collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala index bcc3cee7ebe39..07d10d5ac60e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeometryDataFrameSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.collection.immutable.Seq import org.apache.spark.{SparkIllegalArgumentException, SparkRuntimeException} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -178,4 +179,35 @@ class GeometryDataFrameSuite extends QueryTest with SharedSparkSession { val expectedGeom = Geometry.fromWKB(pointBytes, 0) checkAnswer(df, Seq(Row(expectedGeom))) } + + test("geospatial feature disabled") { + withSQLConf(SQLConf.GEOSPATIAL_ENABLED.key -> "false") { + val geometry = Geometry.fromWKB(point1, 4326) + val schema = StructType(Seq(StructField("col1", GeometryType(4326)))) + // RDD[Row] + schema. + val rdd = sparkContext.parallelize(Seq(Row(geometry))) + checkError( + exception = intercept[AnalysisException] { + spark.createDataFrame(rdd, schema).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + // Java List[Row] + schema. + val javaList = java.util.Arrays.asList(Row(geometry)) + checkError( + exception = intercept[AnalysisException] { + spark.createDataFrame(javaList, schema).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + // Implicit encoder path. + import testImplicits._ + checkError( + exception = intercept[AnalysisException] { + Seq(geometry).toDF("g").collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala index 4f0d567fccc47..323826ca38202 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STExpressionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.st._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -585,4 +586,71 @@ class STExpressionsSuite } } + /** Geospatial feature is disabled. */ + + test("verify that geospatial functions are disabled when the config is off") { + withSQLConf(SQLConf.GEOSPATIAL_ENABLED.key -> "false") { + val dummyArgument = "NULL" + // Verify that SQL ST functions throw the expected exception. + Seq( + s"ST_AsBinary($dummyArgument)", + s"ST_GeogFromWKB($dummyArgument)", + s"ST_GeomFromWKB($dummyArgument)", + s"ST_Srid($dummyArgument)", + s"ST_SetSrid($dummyArgument, $dummyArgument)" + ).foreach { query => + checkError( + exception = intercept[AnalysisException] { + sql(s"SELECT $query").collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + } + } + } + + test("verify that geospatial type casting is disabled when the config is off") { + withSQLConf(SQLConf.GEOSPATIAL_ENABLED.key -> "false") { + // Verify that type casting with geospatial types throws the expected exception. + Seq( + "SELECT NULL::GEOGRAPHY(4326)", + "SELECT NULL::GEOGRAPHY(ANY)", + "SELECT NULL::GEOMETRY(4326)", + "SELECT NULL::GEOMETRY(ANY)" + ).foreach { query => + checkError( + exception = intercept[AnalysisException] { + sql(query).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED") + } + } + } + + test("verify that geospatial type coercion is disabled when the config is off") { + withSQLConf(SQLConf.GEOSPATIAL_ENABLED.key -> "false") { + // Verify that type coercion with geospatial types throws the expected exception. + val value = "NULL" + Seq( + ("GEOGRAPHY(4326)", "GEOGRAPHY(ANY)"), + ("GEOMETRY(4326)", "GEOMETRY(ANY)"), + ("GEOMETRY(ANY)", "GEOMETRY(0)"), + ("GEOMETRY(3857)", "GEOMETRY(4326)") + ).foreach { case (type1, type2) => + val geo1 = s"CAST($value AS $type1)" + val geo2 = s"CAST($value AS $type2)" + Seq( + s"SELECT array($geo1, $geo2)", + s"SELECT nvl($geo1, $geo2)" + ).foreach { query => + checkError( + exception = intercept[AnalysisException] { + sql(query).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED") + } + } + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala index 6aee2cfa0776b..4690a05d67f1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession class STFunctionsSuite extends QueryTest with SharedSparkSession { @@ -58,4 +59,25 @@ class STFunctionsSuite extends QueryTest with SharedSparkSession { Row(4326, 0)) } + /** Geospatial feature is disabled. */ + + test("verify that geospatial functions are disabled when the config is off") { + withSQLConf(SQLConf.GEOSPATIAL_ENABLED.key -> "false") { + val df = Seq[String](null).toDF("col") + Seq( + st_asbinary(lit(null)).as("res"), + st_geogfromwkb(lit(null)).as("res"), + st_geomfromwkb(lit(null)).as("res"), + st_srid(lit(null)).as("res") + ).foreach { func => + checkError( + exception = intercept[AnalysisException] { + df.select(func).collect() + }, + condition = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED" + ) + } + } + } + } From 343e89b301fc6f50b5c31f9ee59d86a96dadf24d Mon Sep 17 00:00:00 2001 From: Ganesha S Date: Wed, 19 Nov 2025 22:24:05 +0800 Subject: [PATCH 168/400] [SPARK-54030][SQL][FOLLOWUP] Refactor test to use withBasicCatalog for consistency ### What changes were proposed in this pull request? This PR refactors the test "corrupted view metadata: mismatch between viewQueryColumnNames and schema" in `SessionCatalogSuite.scala` to use the `withBasicCatalog` helper method instead of manually creating a `SessionCatalog` instance with `newBasicCatalog()`. ### Why are the changes needed? **Consistency**: All other tests in `SessionCatalogSuite` use the `withBasicCatalog` helper pattern ### Does this PR introduce _any_ user-facing change? No. This is a test-only refactoring with no functional changes. ### How was this patch tested? - Verified no linter errors in the modified file - The test logic remains identical, only the catalog initialization pattern changed - Existing test validates the same corrupted view metadata error message ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.5 Closes #53126 from ganeshashree/SPARK-54030-2. Authored-by: Ganesha S Signed-off-by: Wenchen Fan (cherry picked from commit 61668ad02671af0a80ec6d91ddaf02e6b3f042e6) Signed-off-by: Wenchen Fan --- .../catalog/SessionCatalogSuite.scala | 117 +++++++++--------- 1 file changed, 59 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 92a7154c59cb4..a1b7113d79478 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -2030,66 +2030,67 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("corrupted view metadata: mismatch between viewQueryColumnNames and schema") { withSQLConf("spark.sql.viewSchemaBinding.enabled" -> "true") { - val catalog = new SessionCatalog(newBasicCatalog()) - val db = "test_db" - catalog.createDatabase(newDb(db), ignoreIfExists = false) - - // First create a base table for the view to reference - val baseTable = CatalogTable( - identifier = TableIdentifier("base_table", Some(db)), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType() - .add("id", IntegerType) - .add("name", StringType) - .add("value", DoubleType) - ) - catalog.createTable(baseTable, ignoreIfExists = false) - - // Create a view with corrupted metadata where viewQueryColumnNames length - // doesn't match schema length - // We need to set the properties to define viewQueryColumnNames - val properties = Map( - "view.query.out.numCols" -> "2", - "view.query.out.col.0" -> "id", - "view.query.out.col.1" -> "name", - "view.schema.mode" -> "binding" // Ensure it's not SchemaEvolution - ) - val corruptedView = CatalogTable( - identifier = TableIdentifier("corrupted_view", Some(db)), - tableType = CatalogTableType.VIEW, - storage = CatalogStorageFormat.empty, - schema = new StructType() - .add("id", IntegerType) - .add("name", StringType) - .add("value", DoubleType), - viewText = Some("SELECT * FROM test_db.base_table"), - provider = Some("spark"), // Ensure it's not Hive-created - properties = properties // Only 2 query column names but schema has 3 columns - ) - - catalog.createTable(corruptedView, ignoreIfExists = false) - - // Verify the view was created with corrupted metadata - val retrievedView = catalog.getTableMetadata(TableIdentifier("corrupted_view", Some(db))) - assert(retrievedView.viewQueryColumnNames.length == 2) - assert(retrievedView.schema.length == 3) + withBasicCatalog { catalog => + val db = "test_db" + catalog.createDatabase(newDb(db), ignoreIfExists = false) + + // First create a base table for the view to reference + val baseTable = CatalogTable( + identifier = TableIdentifier("base_table", Some(db)), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("id", IntegerType) + .add("name", StringType) + .add("value", DoubleType) + ) + catalog.createTable(baseTable, ignoreIfExists = false) + + // Create a view with corrupted metadata where viewQueryColumnNames length + // doesn't match schema length + // We need to set the properties to define viewQueryColumnNames + val properties = Map( + "view.query.out.numCols" -> "2", + "view.query.out.col.0" -> "id", + "view.query.out.col.1" -> "name", + "view.schema.mode" -> "binding" // Ensure it's not SchemaEvolution + ) + val corruptedView = CatalogTable( + identifier = TableIdentifier("corrupted_view", Some(db)), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("id", IntegerType) + .add("name", StringType) + .add("value", DoubleType), + viewText = Some("SELECT * FROM test_db.base_table"), + provider = Some("spark"), // Ensure it's not Hive-created + properties = properties // Only 2 query column names but schema has 3 columns + ) - // Attempting to look up the view should throw an assertion error with detailed message - val exception = intercept[AssertionError] { - catalog.lookupRelation(TableIdentifier("corrupted_view", Some(db))) + catalog.createTable(corruptedView, ignoreIfExists = false) + + // Verify the view was created with corrupted metadata + val retrievedView = catalog.getTableMetadata(TableIdentifier("corrupted_view", Some(db))) + assert(retrievedView.viewQueryColumnNames.length == 2) + assert(retrievedView.schema.length == 3) + + // Attempting to look up the view should throw an assertion error with detailed message + val exception = intercept[AssertionError] { + catalog.lookupRelation(TableIdentifier("corrupted_view", Some(db))) + } + + // The expected message pattern allows for optional catalog prefix + val expectedPattern = + "assertion failed: Corrupted view metadata detected for view " + + "(\\`\\w+\\`\\.)?\\`test_db\\`\\.\\`corrupted_view\\`\\. " + + "The number of view query column names 2 " + + "does not match the number of columns in the view schema 3\\. " + + "View query column names: \\[id, name\\], " + + "View schema columns: \\[id, name, value\\]\\. " + + "This indicates corrupted view metadata that needs to be repaired\\." + assert(exception.getMessage.matches(expectedPattern)) } - - // The expected message pattern allows for optional catalog prefix - val expectedPattern = - "assertion failed: Corrupted view metadata detected for view " + - "(\\`\\w+\\`\\.)?\\`test_db\\`\\.\\`corrupted_view\\`\\. " + - "The number of view query column names 2 " + - "does not match the number of columns in the view schema 3\\. " + - "View query column names: \\[id, name\\], " + - "View schema columns: \\[id, name, value\\]\\. " + - "This indicates corrupted view metadata that needs to be repaired\\." - assert(exception.getMessage.matches(expectedPattern)) } } From 29f4d44a29bc1b6b35acd725cb6cf8c5c53b8b76 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Thu, 20 Nov 2025 07:08:14 +0900 Subject: [PATCH 169/400] [SPARK-54151][GEO][PYTHON][FOLLOWUP] Update examples for ST functions in PySpark ### What changes were proposed in this pull request? Make a few updates to ST functions in PySpark: - Remove alias `result`. - Separate examples. ### Why are the changes needed? Improve clarity. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests suffice. ``` python/run-tests --testnames pyspark.sql.functions.builtin ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53125 from uros-db/geo-pyspark-st-examples. Authored-by: Uros Bojanic Signed-off-by: Hyukjin Kwon (cherry picked from commit ee0f69245ecdb0b4266268393b5914e2dee8ad61) Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 32 +++++++++++++++---------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index f0fbe1529fea2..76c5e5336a183 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -25917,14 +25917,18 @@ def st_asbinary(geo: "ColumnOrName") -> Column: Examples -------- + + Example 1: Getting WKB from GEOGRAPHY. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa - >>> df.select(sf.hex(sf.st_asbinary(sf.st_geogfromwkb('wkb'))).alias('result')).collect() - [Row(result='0101000000000000000000F03F0000000000000040')] + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geogfromwkb('wkb')))).collect() + [Row(hex(st_asbinary(st_geogfromwkb(wkb)))='0101000000000000000000F03F0000000000000040')] + + Example 2: Getting WKB from GEOMETRY. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa - >>> df.select(sf.hex(sf.st_asbinary(sf.st_geomfromwkb('wkb'))).alias('result')).collect() - [Row(result='0101000000000000000000F03F0000000000000040')] + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geomfromwkb('wkb')))).collect() + [Row(hex(st_asbinary(st_geomfromwkb(wkb)))='0101000000000000000000F03F0000000000000040')] """ return _invoke_function_over_columns("st_asbinary", geo) @@ -25944,8 +25948,8 @@ def st_geogfromwkb(wkb: "ColumnOrName") -> Column: -------- >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa - >>> df.select(sf.hex(sf.st_asbinary(sf.st_geogfromwkb('wkb'))).alias('result')).collect() - [Row(result='0101000000000000000000F03F0000000000000040')] + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geogfromwkb('wkb')))).collect() + [Row(hex(st_asbinary(st_geogfromwkb(wkb)))='0101000000000000000000F03F0000000000000040')] """ return _invoke_function_over_columns("st_geogfromwkb", wkb) @@ -25965,8 +25969,8 @@ def st_geomfromwkb(wkb: "ColumnOrName") -> Column: -------- >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa - >>> df.select(sf.hex(sf.st_asbinary(sf.st_geomfromwkb('wkb'))).alias('result')).collect() - [Row(result='0101000000000000000000F03F0000000000000040')] + >>> df.select(sf.hex(sf.st_asbinary(sf.st_geomfromwkb('wkb')))).collect() + [Row(hex(st_asbinary(st_geomfromwkb(wkb)))='0101000000000000000000F03F0000000000000040')] """ return _invoke_function_over_columns("st_geomfromwkb", wkb) @@ -25984,14 +25988,18 @@ def st_srid(geo: "ColumnOrName") -> Column: Examples -------- + + Example 1: Getting the SRID of GEOGRAPHY. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa - >>> df.select(sf.st_srid(sf.st_geogfromwkb('wkb')).alias('result')).collect() - [Row(result=4326)] + >>> df.select(sf.st_srid(sf.st_geogfromwkb('wkb'))).collect() + [Row(st_srid(st_geogfromwkb(wkb))=4326)] + + Example 2: Getting the SRID of GEOMETRY. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa - >>> df.select(sf.st_srid(sf.st_geomfromwkb('wkb')).alias('result')).collect() - [Row(result=0)] + >>> df.select(sf.st_srid(sf.st_geomfromwkb('wkb'))).collect() + [Row(st_srid(st_geomfromwkb(wkb))=0)] """ return _invoke_function_over_columns("st_srid", geo) From 1b14972f715efe4fd59dcbfa864865ef94103229 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Thu, 20 Nov 2025 07:10:24 +0900 Subject: [PATCH 170/400] [SPARK-54399][GEO][SQL][PYTHON] Implement the st_setsrid function in Scala and PySpark ### What changes were proposed in this pull request? Implement the `st_setsrid` function in Scala and PySpark API. ### Why are the changes needed? Expand API support for the `ST_SetSrid` expression. ### Does this PR introduce _any_ user-facing change? Yes, the new function is now available in Scala and PySpark API. ### How was this patch tested? Added appropriate Scala function unit tests: - `STFunctionsSuite` Added appropriate PySpark function unit tests: - `test_functions` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53117 from uros-db/geo-ST_SetSrid-scala. Authored-by: Uros Bojanic Signed-off-by: Hyukjin Kwon (cherry picked from commit 158a132e62890beee6f3cb4c74dd54ccfc41e877) Signed-off-by: Hyukjin Kwon --- .../reference/pyspark.sql/functions.rst | 1 + .../pyspark/sql/connect/functions/builtin.py | 9 +++++ python/pyspark/sql/functions/__init__.py | 1 + python/pyspark/sql/functions/builtin.py | 33 +++++++++++++++++++ python/pyspark/sql/tests/test_functions.py | 19 +++++++++++ .../org/apache/spark/sql/functions.scala | 18 ++++++++++ .../apache/spark/sql/STFunctionsSuite.scala | 21 +++++++++++- 7 files changed, 101 insertions(+), 1 deletion(-) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index 280465af3e912..f6ce1a3f2d065 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -678,6 +678,7 @@ Geospatial ST Functions st_asbinary st_geogfromwkb st_geomfromwkb + st_setsrid st_srid diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 2bd5b45ed1546..69706398253c0 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -4969,6 +4969,15 @@ def st_geomfromwkb(wkb: "ColumnOrName") -> Column: st_geomfromwkb.__doc__ = pysparkfuncs.st_geomfromwkb.__doc__ +def st_setsrid(geo: "ColumnOrName", srid: Union["ColumnOrName", int]) -> Column: + srid = _enum_to_value(srid) + srid = lit(srid) if isinstance(srid, int) else srid + return _invoke_function_over_columns("st_setsrid", geo, srid) + + +st_setsrid.__doc__ = pysparkfuncs.st_setsrid.__doc__ + + def st_srid(geo: "ColumnOrName") -> Column: return _invoke_function_over_columns("st_srid", geo) diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index 173e074564534..6bbc69dc9bf65 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -544,6 +544,7 @@ "st_asbinary", "st_geogfromwkb", "st_geomfromwkb", + "st_setsrid", "st_srid", # Call Functions "call_udf", diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 76c5e5336a183..12bac34289d60 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -25975,6 +25975,39 @@ def st_geomfromwkb(wkb: "ColumnOrName") -> Column: return _invoke_function_over_columns("st_geomfromwkb", wkb) +@_try_remote_functions +def st_setsrid(geo: "ColumnOrName", srid: Union["ColumnOrName", int]) -> Column: + """Returns a new GEOGRAPHY or GEOMETRY value whose SRID is the specified SRID value. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + geo : :class:`~pyspark.sql.Column` or str + A geospatial value, either a GEOGRAPHY or a GEOMETRY. + srid : :class:`~pyspark.sql.Column` or int + An INTEGER representing the new SRID of the geospatial value. + + Examples + -------- + + Example 1: Setting the SRID on GEOGRAPHY with SRID from another column. + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'), 4326)], ['wkb', 'srid']) # noqa + >>> df.select(sf.st_srid(sf.st_setsrid(sf.st_geogfromwkb('wkb'), 'srid'))).collect() + [Row(st_srid(st_setsrid(st_geogfromwkb(wkb), srid))=4326)] + + Example 2: Setting the SRID on GEOMETRY with SRID as an integer literal. + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(bytes.fromhex('0101000000000000000000F03F0000000000000040'),)], ['wkb']) # noqa + >>> df.select(sf.st_srid(sf.st_setsrid(sf.st_geomfromwkb('wkb'), 4326))).collect() + [Row(st_srid(st_setsrid(st_geomfromwkb(wkb), 4326))=4326)] + """ + srid = _enum_to_value(srid) + srid = lit(srid) if isinstance(srid, int) else srid + return _invoke_function_over_columns("st_setsrid", geo, srid) + + @_try_remote_functions def st_srid(geo: "ColumnOrName") -> Column: """Returns the SRID of the input GEOGRAPHY or GEOMETRY value. diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index a1a4f19e05227..23c895f5629e0 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -2974,6 +2974,25 @@ def test_st_asbinary(self): ) self.assertEqual(results, [expected]) + def test_st_setsrid(self): + df = self.spark.createDataFrame( + [(bytes.fromhex("0101000000000000000000F03F0000000000000040"), 4326)], + ["wkb", "srid"], + ) + results = df.select( + F.st_srid(F.st_setsrid(F.st_geogfromwkb("wkb"), "srid")), + F.st_srid(F.st_setsrid(F.st_geomfromwkb("wkb"), "srid")), + F.st_srid(F.st_setsrid(F.st_geogfromwkb("wkb"), 4326)), + F.st_srid(F.st_setsrid(F.st_geomfromwkb("wkb"), 4326)), + ).collect() + expected = Row( + 4326, + 4326, + 4326, + 4326, + ) + self.assertEqual(results, [expected]) + def test_st_srid(self): df = self.spark.createDataFrame( [(bytes.fromhex("0101000000000000000000F03F0000000000000040"),)], diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index 6f56c86541e43..0a14491de2238 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -9471,6 +9471,24 @@ object functions { def st_geomfromwkb(wkb: Column): Column = Column.fn("st_geomfromwkb", wkb) + /** + * Returns a new GEOGRAPHY or GEOMETRY value whose SRID is the specified SRID value. + * + * @group st_funcs + * @since 4.1.0 + */ + def st_setsrid(geo: Column, srid: Column): Column = + Column.fn("st_setsrid", geo, srid) + + /** + * Returns a new GEOGRAPHY or GEOMETRY value whose SRID is the specified SRID value. + * + * @group st_funcs + * @since 4.1.0 + */ + def st_setsrid(geo: Column, srid: Int): Column = + Column.fn("st_setsrid", geo, lit(srid)) + /** * Returns the SRID of the input GEOGRAPHY or GEOMETRY value. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala index 4690a05d67f1c..49ff4ea37bc47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/STFunctionsSuite.scala @@ -59,6 +59,24 @@ class STFunctionsSuite extends QueryTest with SharedSparkSession { Row(4326, 0)) } + /** ST modifier expressions. */ + + test("st_setsrid") { + // Test data: Well-Known Binary (WKB) representations. + val df = Seq[(String, Int)]( + ( + "0101000000000000000000f03f0000000000000040", 4326 + )).toDF("wkb", "srid") + // ST_GeogFromWKB/ST_GeomFromWKB and ST_Srid. + checkAnswer( + df.select( + st_srid(st_setsrid(st_geogfromwkb(unhex($"wkb")), $"srid")).as("col0"), + st_srid(st_setsrid(st_geomfromwkb(unhex($"wkb")), $"srid")).as("col1"), + st_srid(st_setsrid(st_geomfromwkb(unhex($"wkb")), 4326)).as("col1"), + st_srid(st_setsrid(st_geomfromwkb(unhex($"wkb")), 4326)).as("col1")), + Row(4326, 4326, 4326, 4326)) + } + /** Geospatial feature is disabled. */ test("verify that geospatial functions are disabled when the config is off") { @@ -68,7 +86,8 @@ class STFunctionsSuite extends QueryTest with SharedSparkSession { st_asbinary(lit(null)).as("res"), st_geogfromwkb(lit(null)).as("res"), st_geomfromwkb(lit(null)).as("res"), - st_srid(lit(null)).as("res") + st_srid(lit(null)).as("res"), + st_setsrid(lit(null), lit(null)).as("res") ).foreach { func => checkError( exception = intercept[AnalysisException] { From 25d5af250427db8af8d2f307772e5eeed54d8bde Mon Sep 17 00:00:00 2001 From: Chirag Singh Date: Wed, 19 Nov 2025 14:40:23 -0800 Subject: [PATCH 171/400] [SPARK-53322][SQL][4.1] Select a KeyGroupedShuffleSpec only when join key positions can be fully pushed down ### What changes were proposed in this pull request? When a KeyGroupedShuffleSpec is used to shuffle another child of a JOIN, we must be able to push down JOIN keys or partition values to be able to ensure that both children have matching partitioning. If one child reports a KeyGroupedPartitioning but we can't push down these values (for example, if the child was a key-grouped scan that was checkpointed), then this information cannot be pushed down to the child scan and we should avoid using this shuffle spec to shuffle other children. ### Why are the changes needed? Prevents potential correctness issue when key-grouped partitioning is used on a checkpointed RDD. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? See test changes. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53132 from chirag-s-db/checkpoint-pushdown-4.1. Authored-by: Chirag Singh Signed-off-by: Dongjoon Hyun --- .../exchange/EnsureRequirements.scala | 31 ++++ .../KeyGroupedPartitioningSuite.scala | 145 ++++++++++++++++++ .../exchange/EnsureRequirementsSuite.scala | 75 +++++---- 3 files changed, 218 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index b97d765afcf79..088ece6554c96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -140,6 +140,13 @@ case class EnsureRequirements( // Choose all the specs that can be used to shuffle other children val candidateSpecs = specs .filter(_._2.canCreatePartitioning) + .filter { + // To choose a KeyGroupedShuffleSpec, we must be able to push down SPJ parameters into + // the scan (for join key positions). If these parameters can't be pushed down, this + // spec can't be used to shuffle other children. + case (idx, _: KeyGroupedShuffleSpec) => canPushDownSPJParamsToScan(children(idx)) + case _ => true + } .filter(p => !shouldConsiderMinParallelism || children(p._1).outputPartitioning.numPartitions >= conf.defaultNumShufflePartitions) val bestSpecOpt = if (candidateSpecs.isEmpty) { @@ -402,6 +409,24 @@ case class EnsureRequirements( } } + /** + * Whether SPJ params can be pushed down to the leaf nodes of a physical plan. For a plan to be + * eligible for SPJ parameter pushdown, all leaf nodes must be a KeyGroupedPartitioning-aware + * scan. + * + * Notably, if the leaf of `plan` is an [[RDDScanExec]] created by checkpointing a DSv2 scan, the + * reported partitioning will be a [[KeyGroupedPartitioning]], but this plan will _not_ be + * eligible for SPJ parameter pushdown (as the partitioning is static and can't be easily + * re-grouped or padded with empty partitions according to the partition values on the other side + * of the join). + */ + private def canPushDownSPJParamsToScan(plan: SparkPlan): Boolean = { + plan.collectLeaves().forall { + case _: KeyGroupedPartitionedScan[_] => true + case _ => false + } + } + /** * Checks whether two children, `left` and `right`, of a join operator have compatible * `KeyGroupedPartitioning`, and can benefit from storage-partitioned join. @@ -413,6 +438,12 @@ case class EnsureRequirements( left: SparkPlan, right: SparkPlan, requiredChildDistribution: Seq[Distribution]): Option[Seq[SparkPlan]] = { + // If SPJ params can't be pushed down to either the left or right side, it's unsafe to do an + // SPJ. + if (!canPushDownSPJParamsToScan(left) || !canPushDownSPJParamsToScan(right)) { + return None + } + parent match { case smj: SortMergeJoinExec => checkKeyGroupCompatible(left, right, smj.joinType, requiredChildDistribution) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index c73e8e16fbbb0..aba866e96b5c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.functions.{col, max} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.types._ @@ -2626,4 +2627,148 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { assert(scans.forall(_.inputRDD.partitions.length == 2)) } } + + test("SPARK-53322: checkpointed scans avoid shuffles for aggregates") { + withTempDir { dir => + spark.sparkContext.setCheckpointDir(dir.getPath) + val itemsPartitions = Array(identity("id")) + createTable(items, itemsColumns, itemsPartitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val scanDF = spark.read.table(s"testcat.ns.$items").checkpoint() + val df = scanDF.groupBy("id").agg(max("price").as("res")).select("res") + checkAnswer(df.sort("res"), Seq(Row(10.0), Row(15.5), Row(41.0))) + + val shuffles = collectAllShuffles(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, + "should not contain shuffle when not grouping by partition values") + } + } + + test("SPARK-53322: checkpointed scans aren't used for SPJ") { + withTempDir { dir => + spark.sparkContext.setCheckpointDir(dir.getPath) + val itemsPartitions = Array(identity("id")) + createTable(items, itemsColumns, itemsPartitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 41.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-01-03' as timestamp))") + + val purchase_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchase_partitions) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 40.0, cast('2020-01-01' as timestamp)), " + + s"(3, 25.5, cast('2020-01-03' as timestamp)), " + + s"(4, 20.0, cast('2020-01-04' as timestamp))") + + for { + pushdownValues <- Seq(true, false) + checkpointBothScans <- Seq(true, false) + } { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushdownValues.toString) { + val scanDF1 = spark.read.table(s"testcat.ns.$items").checkpoint().as("i") + val scanDF2 = if (checkpointBothScans) { + spark.read.table(s"testcat.ns.$purchases").checkpoint().as("p") + } else { + spark.read.table(s"testcat.ns.$purchases").as("p") + } + + val df = scanDF1 + .join(scanDF2, col("id") === col("item_id")) + .selectExpr("id", "name", "i.price AS purchase_price", "p.price AS sale_price") + .orderBy("id", "purchase_price", "sale_price") + checkAnswer( + df, + Seq(Row(1, "aa", 41.0, 40.0), Row(3, "cc", 15.5, 25.5)) + ) + // 1 shuffle for SORT and 2 shuffles for JOIN are expected. + assert(collectAllShuffles(df.queryExecution.executedPlan).length === 3) + } + } + } + } + + test("SPARK-53322: checkpointed scans can't shuffle other children on SPJ") { + withTempDir { dir => + spark.sparkContext.setCheckpointDir(dir.getPath) + val itemsPartitions = Array(identity("id")) + createTable(items, itemsColumns, itemsPartitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 41.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-01-03' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 40.0, cast('2020-01-01' as timestamp)), " + + s"(3, 25.5, cast('2020-01-03' as timestamp)), " + + s"(4, 20.0, cast('2020-01-04' as timestamp))") + + Seq(true, false).foreach { pushdownValues => + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushdownValues.toString) { + val scanDF1 = spark.read.table(s"testcat.ns.$items").checkpoint().as("i") + val scanDF2 = spark.read.table(s"testcat.ns.$purchases").as("p") + + val df = scanDF1 + .join(scanDF2, col("id") === col("item_id")) + .selectExpr("id", "name", "i.price AS purchase_price", "p.price AS sale_price") + .orderBy("id", "purchase_price", "sale_price") + checkAnswer( + df, + Seq(Row(1, "aa", 41.0, 40.0), Row(3, "cc", 15.5, 25.5)) + ) + // 1 shuffle for SORT and 2 shuffles for JOIN are expected. + assert(collectAllShuffles(df.queryExecution.executedPlan).length === 3) + } + } + } + } + + test("SPARK-53322: checkpointed scans can be shuffled by children on SPJ") { + withTempDir { dir => + spark.sparkContext.setCheckpointDir(dir.getPath) + val itemsPartitions = Array(identity("id")) + createTable(items, itemsColumns, itemsPartitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 41.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-01-03' as timestamp))") + + createTable(purchases, purchasesColumns, Array(identity("item_id"))) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 40.0, cast('2020-01-01' as timestamp)), " + + s"(3, 25.5, cast('2020-01-03' as timestamp)), " + + s"(4, 20.0, cast('2020-01-04' as timestamp))") + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> "true") { + val scanDF1 = spark.read.table(s"testcat.ns.$items").checkpoint().as("i") + val scanDF2 = spark.read.table(s"testcat.ns.$purchases").as("p") + + val df = scanDF1 + .join(scanDF2, col("id") === col("item_id")) + .selectExpr("id", "name", "i.price AS purchase_price", "p.price AS sale_price") + .orderBy("id", "purchase_price", "sale_price") + checkAnswer( + df, + Seq(Row(1, "aa", 41.0, 40.0), Row(3, "cc", 15.5, 25.5)) + ) + + // One shuffle for the sort and one shuffle for one side of the JOIN are expected. + assert(collectAllShuffles(df.queryExecution.executedPlan).length === 2) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index b94ca46736417..1cc0d795d74f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.statsEstimation.StatsTestPlan import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.execution.{DummySparkPlan, SortExec} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.python.FlatMapCoGroupsInPandasExec import org.apache.spark.sql.execution.window.WindowExec @@ -92,11 +93,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { test("reorder should handle KeyGroupedPartitioning") { // partitioning on the left - val plan1 = DummySparkPlan( + val plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(Seq( years(exprA), bucket(4, exprB), days(exprC)), 4) ) - val plan2 = DummySparkPlan( + val plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(Seq( years(exprB), bucket(4, exprA), days(exprD)), 4) ) @@ -114,7 +115,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // partitioning on the right - val plan3 = DummySparkPlan( + val plan3 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(Seq( bucket(4, exprD), days(exprA), years(exprC)), 4) ) @@ -778,9 +779,9 @@ class EnsureRequirementsSuite extends SharedSparkSession { test("Check with KeyGroupedPartitioning") { // simplest case: identity transforms - var plan1 = DummySparkPlan( - outputPartitioning = KeyGroupedPartitioning(exprA :: exprB :: Nil, 5)) - var plan2 = DummySparkPlan( + var plan1 = new DummySparkPlanWithBatchScanChild( + KeyGroupedPartitioning(exprA :: exprB :: Nil, 5)) + var plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(exprA :: exprC :: Nil, 5)) var smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprA :: exprC :: Nil, Inner, None, plan1, plan2) @@ -794,11 +795,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // matching bucket transforms from both sides - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(16, exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(16, exprC) :: Nil, 4) ) @@ -814,11 +815,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // partition collections - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(16, exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = PartitioningCollection(Seq( KeyGroupedPartitioning(bucket(4, exprA) :: bucket(16, exprC) :: Nil, 4), KeyGroupedPartitioning(bucket(4, exprA) :: bucket(16, exprC) :: Nil, 4)) @@ -844,10 +845,10 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // bucket + years transforms from both sides - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(bucket(4, exprA) :: years(exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(bucket(4, exprA) :: years(exprC) :: Nil, 4) ) smjExec = SortMergeJoinExec( @@ -863,11 +864,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { // by default spark.sql.requireAllClusterKeysForCoPartition is true, so when there isn't // exact match on all partition keys, Spark will fallback to shuffle. - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(4, exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(4, exprC) :: Nil, 4) ) @@ -884,11 +885,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { } test(s"KeyGroupedPartitioning with ${REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key} = false") { - var plan1 = DummySparkPlan( + var plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprB) :: years(exprC) :: Nil, 4) ) - var plan2 = DummySparkPlan( + var plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprC) :: years(exprB) :: Nil, 4) ) @@ -906,11 +907,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // should also work with distributions with duplicated keys - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: years(exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: years(exprC) :: Nil, 4) ) @@ -926,10 +927,10 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // both partitioning and distribution have duplicated keys - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( years(exprA) :: bucket(4, exprB) :: days(exprA) :: Nil, 5)) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( years(exprA) :: bucket(4, exprC) :: days(exprA) :: Nil, 5)) smjExec = SortMergeJoinExec( @@ -944,11 +945,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // invalid case: partitioning key positions don't match - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(4, exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprB) :: bucket(4, exprC) :: Nil, 4) ) @@ -965,11 +966,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // invalid case: different number of buckets (we don't support coalescing/repartitioning yet) - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(4, exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( bucket(4, exprA) :: bucket(8, exprC) :: Nil, 4) ) @@ -985,10 +986,10 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // invalid case: partition key positions match but with different transforms - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(years(exprA) :: bucket(4, exprB) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(days(exprA) :: bucket(4, exprC) :: Nil, 4) ) smjExec = SortMergeJoinExec( @@ -1004,11 +1005,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { // invalid case: multiple references in transform - plan1 = DummySparkPlan( + plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( years(exprA) :: buckets(4, Seq(exprB, exprC)) :: Nil, 4) ) - plan2 = DummySparkPlan( + plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning( years(exprA) :: buckets(4, Seq(exprB, exprC)) :: Nil, 4) ) @@ -1030,11 +1031,11 @@ class EnsureRequirementsSuite extends SharedSparkSession { val rightPartValues = Seq(Array[Any](1, 1), Array[Any](2, 2), Array[Any](3, 3)) .map(new GenericInternalRow(_)) - var plan1 = DummySparkPlan( + var plan1 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(bucket(4, exprB) :: bucket(8, exprC) :: Nil, leftPartValues.length, leftPartValues) ) - var plan2 = DummySparkPlan( + var plan2 = new DummySparkPlanWithBatchScanChild( outputPartitioning = KeyGroupedPartitioning(bucket(4, exprC) :: bucket(8, exprB) :: Nil, rightPartValues.length, rightPartValues) ) @@ -1052,7 +1053,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // With partition collections - plan1 = DummySparkPlan(outputPartitioning = + plan1 = new DummySparkPlanWithBatchScanChild(outputPartitioning = PartitioningCollection( Seq(KeyGroupedPartitioning(bucket(4, exprB) :: bucket(8, exprC) :: Nil, leftPartValues.length, leftPartValues), @@ -1076,7 +1077,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } // Nested partition collections - plan2 = DummySparkPlan(outputPartitioning = + plan2 = new DummySparkPlanWithBatchScanChild(outputPartitioning = PartitioningCollection( Seq( PartitioningCollection( @@ -1119,7 +1120,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { val a1 = AttributeReference("a1", IntegerType)() val partitionValue = Seq(50, 51, 52).map(v => InternalRow.fromSeq(Seq(v))) - val plan1 = DummySparkPlan(outputPartitioning = KeyGroupedPartitioning( + val plan1 = new DummySparkPlanWithBatchScanChild(outputPartitioning = KeyGroupedPartitioning( identity(a1) :: Nil, 4, partitionValue)) val plan2 = DummySparkPlan(outputPartitioning = SinglePartition) @@ -1389,4 +1390,12 @@ class EnsureRequirementsSuite extends SharedSparkSession { def days(expr: Expression): TransformExpression = { TransformExpression(DaysFunction, Seq(expr)) } + + private class DummySparkPlanWithBatchScanChild(outputPartitioning: Partitioning) + extends DummySparkPlan( + children = Seq(BatchScanExec(Seq.empty, null, Seq.empty, table = null)), + outputPartitioning = outputPartitioning, + requiredChildDistribution = Seq(UnspecifiedDistribution), + requiredChildOrdering = Seq(Seq.empty) + ) } From f9b2ef8c62f0bf038196e763664bbef4ff1efbfb Mon Sep 17 00:00:00 2001 From: Jim Halfpenny Date: Wed, 19 Nov 2025 14:45:00 -0800 Subject: [PATCH 172/400] [SPARK-54416][DOCS] Remove references to Pod Security Policies from Kubernetes docs ### What changes were proposed in this pull request? The Spark documentation references the use of Pod Security Policies to restrict the privileges of Spark pods on Kubernetes. PodSecurityPolicy was [deprecated](https://kubernetes.io/blog/2021/04/08/kubernetes-1-21-release-announcement/#podsecuritypolicy-deprecation) in Kubernetes v1.21, and removed from Kubernetes in v1.25. The documentation should be updated to remove this reference and to direct users to the currently maintained Pod Security Admission Controller docs instead. ### Why are the changes needed? Maintenance of the documentation to point to relevant and currently supported security features in Kubernetes. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Documentation changes only, no function changes to Spark. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53130 from Jimvin/master. Authored-by: Jim Halfpenny Signed-off-by: Dongjoon Hyun (cherry picked from commit f328b5ef14c9ef4e2d04ab69c0578ab461388715) Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index bdc2a1a156b0e..e253ee4225407 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -34,13 +34,13 @@ Please see [Spark Security](security.html) and the specific security sections in Images built from the project provided Dockerfiles contain a default [`USER`](https://docs.docker.com/engine/reference/builder/#user) directive with a default UID of `185`. This means that the resulting images will be running the Spark processes as this UID inside the container. Security conscious deployments should consider providing custom images with `USER` directives specifying their desired unprivileged UID and GID. The resulting UID should include the root group in its supplementary groups in order to be able to run the Spark executables. Users building their own images with the provided `docker-image-tool.sh` script can use the `-u ` option to specify the desired UID. -Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. This can be used to override the `USER` directives in the images themselves. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. +Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. This can be used to override the `USER` directives in the images themselves. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use the [Pod Security Admission Controller](https://kubernetes.io/docs/concepts/security/pod-security-admission/) if they wish to limit the users that pods may run as. ## Volume Mounts As described later in this document under [Using Kubernetes Volumes](#using-kubernetes-volumes) Spark on K8S provides configuration options that allow for mounting certain volume types into the driver and executor pods. In particular it allows for [`hostPath`](https://kubernetes.io/docs/concepts/storage/volumes/#hostpath) volumes which as described in the Kubernetes documentation have known security vulnerabilities. -Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/) to limit the ability to mount `hostPath` volumes appropriately for their environments. +Cluster administrators should use the [Pod Security Admission Controller](https://kubernetes.io/docs/concepts/security/pod-security-admission/) to limit the ability to mount `hostPath` volumes appropriately for their environments. # Prerequisites From 1016663bf034d7d999b21ff3a93ca8c1e2358cdc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 19 Nov 2025 15:48:48 -0800 Subject: [PATCH 173/400] [SPARK-54426][INFRA] Fix `release-build.sh` to detect `REPO_ID` correctly ### What changes were proposed in this pull request? This PR aims to fix `release-build.sh` to detect `REPO_ID` correctly. ### Why are the changes needed? Previously, we use `grep -A 5` to find `description` tag. https://github.com/apache/spark/blob/f328b5ef14c9ef4e2d04ab69c0578ab461388715/dev/create-release/release-build.sh#L501-L502 However, it's insufficient as of now. According to Today's result, we need to grep 13 lines like the following. ``` $ curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_PASSWORD" https://repository.apache.org/service/local/staging/profile_repositories | grep -A 5 "orgapachespark-" orgapachespark-1505 closed release dongjoon curl/7.81.0 35.94.112.49 ``` ``` $ curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_PASSWORD" https://repository.apache.org/service/local/staging/profile_repositories | grep -A 13 "orgapachespark-" orgapachespark-1505 closed release dongjoon curl/7.81.0 35.94.112.49 https://repository.apache.org/content/repositories/orgapachespark-1505 2025-11-16T20:23:35.413Z Sun Nov 16 20:23:35 UTC 2025 1763324615413 2025-11-16T21:02:45.041Z Sun Nov 16 21:02:45 UTC 2025 1763326965041 Apache Spark 4.1.0-preview4 (commit c125aea395b) ``` ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Manually test. ``` $ curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_PASSWORD" https://repository.apache.org/service/local/staging/profile_repositories | grep -A 13 "orgapachespark-" | awk '// { id = $0 } // && $0 ~ /Apache Spark '"$RELEASE_VERSION"'/ { print id }' orgapachespark-1505 ``` After merging this, I'm going to test with `Apache Spark 4.1.0-preview4`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53136 from dongjoon-hyun/SPARK-54426. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit be281fba3a1b251ff82248af82877d7aec29ba60) Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index b984876f41643..21f0d052047b3 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -498,7 +498,7 @@ EOF # Find latest orgapachespark-* repo for this release version REPO_ID=$(curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_PASSWORD" \ https://repository.apache.org/service/local/staging/profile_repositories | \ - grep -A 5 "orgapachespark-" | \ + grep -A 13 "orgapachespark-" | \ awk '// { id = $0 } // && $0 ~ /Apache Spark '"$RELEASE_VERSION"'/ { print id }' | \ grep -oP '(?<=)orgapachespark-[0-9]+(?=)' | \ sort -V | tail -n 1) From ccdd720e599b29d1d5fc8c6487494f20119b31b0 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 19 Nov 2025 21:35:27 -0800 Subject: [PATCH 174/400] [SPARK-54358][SDP][FOLLOWUP] Add `tableIdentifierToPathString` helper method for SDP checkpoint path construction ### What changes were proposed in this pull request? Followups from https://github.com/apache/spark/pull/53070 to improve code clarity. ### Why are the changes needed? Make sure the code for constructing SDP checkpoint directory paths is clear. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #53089 from sryza/collide-followups. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 10d1b4c383590769c4b8b6e1aa8e1162c4452600) Signed-off-by: Dongjoon Hyun --- .../spark/sql/pipelines/graph/SystemMetadata.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala index e3c93f97a54d5..c2c8f7f0c10b2 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SystemMetadata.scala @@ -22,6 +22,7 @@ import scala.util.Try import org.apache.hadoop.fs.Path import org.apache.spark.internal.{Logging, LogKeys} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.classic.SparkSession sealed trait SystemMetadata {} @@ -44,7 +45,9 @@ case class FlowSystemMetadata( Option(if (graph.table.contains(flow.destinationIdentifier) || graph.sink.contains(flow.destinationIdentifier)) { val checkpointRoot = new Path(context.storageRoot, "_checkpoints") - val flowTableId = flow.destinationIdentifier.nameParts.mkString(Path.SEPARATOR) + // Different tables in the pipeline can have flows with the same name, so we include + // the table's fully qualified identifier in the path to avoid collisions. + val flowTableId = tableIdentifierToPathString(flow.destinationIdentifier) val flowName = flow.identifier.table val checkpointDir = new Path( new Path(checkpointRoot, flowTableId), @@ -62,6 +65,13 @@ case class FlowSystemMetadata( }) } + /** + * Converts a TableIdentifier to a path string by joining its name parts with the path separator. + */ + private def tableIdentifierToPathString(tableIdentifier: TableIdentifier): String = { + tableIdentifier.nameParts.mkString(Path.SEPARATOR) + } + /** Returns the location for the most recent checkpoint of a given flow. */ def latestCheckpointLocation: String = { val checkpointsDir = flowCheckpointsDirOpt().get From 7fcfdf30a18d1628d09c24e585d36fb0648253fa Mon Sep 17 00:00:00 2001 From: Tengfei Huang Date: Wed, 19 Nov 2025 21:47:02 -0800 Subject: [PATCH 175/400] [SPARK-54312][CORE] Avoid repeatedly scheduling tasks for SendHeartbeat/WorkDirClean in standalone worker ### What changes were proposed in this pull request? Currently, [worker](https://github.com/apache/spark/blob/87b3b94232436528f88c9a7aa7ee70758b85a33a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala#L495) will schedule tasks forwarding `SendHeartbeat` and `WorkDirCleanup` while `handleRegisterResponse`. While worker registration could happen multiple times in case of heartbeat timeout/disconnected from master, in these cases the tasks would be scheduled multiple times. To fix the issue: - Adding `heartbeatTask` and `workDirCleanupTask` in worker to tell whether these tasks have been scheduled - `heartbeatTask` and `workDirCleanupTask` will be initialized after the 1st registration, and then skipped scheduling these tasks in later registration. - Cancel the task and reset `heartbeatTask` and `workDirCleanupTask` when worker stops. ### Why are the changes needed? Fix the issue repeatedly scheduling SendHeartbeat/WorkDirClean tasks after worker registration. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT added ### Was this patch authored or co-authored using generative AI tooling? No Closes #53054 from ivoson/duplicate-worker-heartbeat. Authored-by: Tengfei Huang Signed-off-by: Dongjoon Hyun (cherry picked from commit d51b4331e419d56822990846ba1df0efdbf0b6b2) Signed-off-by: Dongjoon Hyun --- .../apache/spark/deploy/worker/Worker.scala | 27 +++++++++--- .../spark/deploy/worker/WorkerSuite.scala | 42 ++++++++++++++++++- 2 files changed, 61 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 98da33a429eca..93fb64f485f62 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -211,6 +211,8 @@ private[deploy] class Worker( private var registerMasterFutures: Array[JFuture[_]] = null private var registrationRetryTimer: Option[JScheduledFuture[_]] = None + private var heartbeatTask: Option[JScheduledFuture[_]] = None + private var workDirCleanupTask: Option[JScheduledFuture[_]] = None // A thread pool for registering with masters. Because registering with a master is a blocking // action, this thread pool must be able to create "masterRpcAddresses.size" threads at the same @@ -492,16 +494,25 @@ private[deploy] class Worker( logInfo(log"Successfully registered with master ${MDC(MASTER_URL, preferredMasterAddress)}") registered = true changeMaster(masterRef, masterWebUiUrl, masterAddress) - forwardMessageScheduler.scheduleAtFixedRate( - () => Utils.tryLogNonFatalError { self.send(SendHeartbeat) }, - 0, HEARTBEAT_MILLIS, TimeUnit.MILLISECONDS) - if (CLEANUP_ENABLED) { + + // Only schedule heartbeat task if not already scheduled. The existing task will + // continue running through reconnections, and the SendHeartbeat handler already + // checks the 'connected' flag before sending heartbeats to master. + if (heartbeatTask.isEmpty) { + heartbeatTask = Some(forwardMessageScheduler.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { + self.send(SendHeartbeat) + }, + 0, HEARTBEAT_MILLIS, TimeUnit.MILLISECONDS)) + } + // Only schedule work directory cleanup task if not already scheduled + if (CLEANUP_ENABLED && workDirCleanupTask.isEmpty) { logInfo( log"Worker cleanup enabled; old application directories will be deleted in: " + log"${MDC(PATH, workDir)}") - forwardMessageScheduler.scheduleAtFixedRate( + workDirCleanupTask = Some(forwardMessageScheduler.scheduleAtFixedRate( () => Utils.tryLogNonFatalError { self.send(WorkDirCleanup) }, - CLEANUP_INTERVAL_MILLIS, CLEANUP_INTERVAL_MILLIS, TimeUnit.MILLISECONDS) + CLEANUP_INTERVAL_MILLIS, CLEANUP_INTERVAL_MILLIS, TimeUnit.MILLISECONDS)) } val execs = executors.values.map { e => @@ -852,6 +863,10 @@ private[deploy] class Worker( cleanupThreadExecutor.shutdownNow() metricsSystem.report() cancelLastRegistrationRetry() + heartbeatTask.foreach(_.cancel(true)) + heartbeatTask = None + workDirCleanupTask.foreach(_.cancel(true)) + workDirCleanupTask = None forwardMessageScheduler.shutdownNow() registerMasterThreadPool.shutdownNow() executors.values.foreach(_.kill()) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index ff5d314d1688a..f9a0efce88708 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.worker import java.io.{File, IOException} +import java.util.concurrent.{ScheduledFuture => JScheduledFuture} import java.util.concurrent.atomic.AtomicBoolean import java.util.function.Supplier @@ -37,7 +38,7 @@ import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.TestUtils.{createTempJsonFile, createTempScriptWithExpectedOutput} import org.apache.spark.deploy.{Command, ExecutorState, ExternalShuffleService} -import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged, WorkDirCleanup} +import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged, RegisteredWorker, WorkDirCleanup} import org.apache.spark.deploy.master.DriverState import org.apache.spark.internal.config import org.apache.spark.internal.config.SHUFFLE_SERVICE_DB_BACKEND @@ -46,7 +47,7 @@ import org.apache.spark.network.shuffledb.DBBackend import org.apache.spark.resource.{ResourceAllocation, ResourceInformation} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs.{WORKER_FPGA_ID, WORKER_GPU_ID} -import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.util.Utils class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { @@ -405,4 +406,41 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter with P }.getMessage assert(m.contains("Whitespace is not allowed")) } + + test("SPARK-54312: heartbeat task and workdir cleanup task should only be scheduled once " + + "across multiple registrations") { + val worker = spy(makeWorker()) + val masterWebUiUrl = "https://1.2.3.4:8080" + val masterAddress = RpcAddress("1.2.3.4", 1234) + val masterRef = mock(classOf[RpcEndpointRef]) + when(masterRef.address).thenReturn(masterAddress) + + def getHeartbeatTask(worker: Worker): Option[JScheduledFuture[_]] = { + val _heartbeatTask = + PrivateMethod[Option[JScheduledFuture[_]]](Symbol("heartbeatTask")) + worker.invokePrivate(_heartbeatTask()) + } + + def getWorkDirCleanupTask(worker: Worker): Option[JScheduledFuture[_]] = { + val _workDirCleanupTask = + PrivateMethod[Option[JScheduledFuture[_]]](Symbol("workDirCleanupTask")) + worker.invokePrivate(_workDirCleanupTask()) + } + + // Tasks should not be scheduled yet before registration + assert(getHeartbeatTask(worker).isEmpty && getWorkDirCleanupTask(worker).isEmpty) + + val msg = RegisteredWorker(masterRef, masterWebUiUrl, masterAddress, duplicate = false) + // Simulate first registration - this should schedule both tasks + worker.receive(msg) + val heartbeatTask = getHeartbeatTask(worker) + val workDirCleanupTask = getWorkDirCleanupTask(worker) + assert(heartbeatTask.isDefined && workDirCleanupTask.isDefined) + + // Simulate disconnection and re-registration + worker.receive(msg) + // After re-registration, the task references should be the same (not rescheduled) + assert(getHeartbeatTask(worker) == heartbeatTask) + assert(getWorkDirCleanupTask(worker) == workDirCleanupTask) + } } From 0c0802335d4d9712c013aac6c79b6a94ce9b9f0c Mon Sep 17 00:00:00 2001 From: Mihailo Timotic Date: Thu, 20 Nov 2025 23:52:37 +0800 Subject: [PATCH 176/400] [SPARK-54417][SQL] Fix error message for scalar subquery in IDENTIFIER clause ### What changes were proposed in this pull request? Fix error message for scalar subquery in IDENTIFIER clause ### Why are the changes needed? Previously, for a query like: `SELECT * FROM IDENTIFIER((SELECT 'tableName'))` we would throw an `INTERNAL_ERROR` because we never call `IdentifierResolution.evalIdentifierExpr` to check if the identifier expression is foldable. The call never happens, because in order to call `eval`, the expression has to be resolved first, but the subquery has `UnresolvedAlias('tableName')` that never gets resolved. In `CheckAnalysis`, if the identifier expression is not resolved, I propose that we throw `NOT_A_CONSTANT_STRING.NOT_CONSTANT`, because ALL identifier expressions need to be constant and if by the end of analysis the expression is not resolved, that means it wasn't constant. ### Does this PR introduce _any_ user-facing change? Error message improvement. ### How was this patch tested? Added new test cases to golden file. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53133 from mihailotim-db/mihailotim-db/fix_error_message_ident. Authored-by: Mihailo Timotic Signed-off-by: Wenchen Fan (cherry picked from commit 3dcb661df7c0f2a7786eb06065f1ef30f805f2be) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 + .../identifier-clause-legacy.sql.out | 118 ++++++++++++++++ .../identifier-clause.sql.out | 118 ++++++++++++++++ .../sql-tests/inputs/identifier-clause.sql | 8 ++ .../results/identifier-clause-legacy.sql.out | 131 ++++++++++++++++++ .../results/identifier-clause.sql.out | 131 ++++++++++++++++++ 6 files changed, 512 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 2ff842553bee6..04451162276fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -340,6 +340,12 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString errorClass = "UNSUPPORTED_FEATURE.OVERWRITE_BY_SUBQUERY", messageParameters = Map.empty) + case p: PlanWithUnresolvedIdentifier if !p.identifierExpr.resolved => + p.identifierExpr.failAnalysis( + errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + messageParameters = Map("name" -> "IDENTIFIER", "expr" -> p.identifierExpr.sql) + ) + case operator: LogicalPlan => operator transformExpressionsDown { case hof: HigherOrderFunction if hof.arguments.exists { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out index 17fcc9b47729e..2d7d551f51ca5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -735,6 +735,124 @@ org.apache.spark.sql.AnalysisException } +-- !query +CREATE TABLE t(col1 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT * FROM IDENTIFIER((SELECT 't')) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 37, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT * FROM (SELECT IDENTIFIER((SELECT 'col1')) FROM IDENTIFIER((SELECT 't'))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 78, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1')) FROM VALUES(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 33, + "fragment" : "(SELECT 'col1')" + } ] +} + + +-- !query +SELECT col1, IDENTIFIER((SELECT col1)) FROM VALUES(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery(col1)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 37, + "fragment" : "(SELECT col1)" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1', 'col2')) FROM VALUES(1,2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\", \"TIME\"", + "unsupportedType" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 32, + "fragment" : "SELECT 'col1'" + } ] +} + + +-- !query +DROP TABLE t +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + -- !query CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index fc8d225e22714..9adecd5e53733 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -735,6 +735,124 @@ org.apache.spark.sql.AnalysisException } +-- !query +CREATE TABLE t(col1 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT * FROM IDENTIFIER((SELECT 't')) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 37, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT * FROM (SELECT IDENTIFIER((SELECT 'col1')) FROM IDENTIFIER((SELECT 't'))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 78, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1')) FROM VALUES(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 33, + "fragment" : "(SELECT 'col1')" + } ] +} + + +-- !query +SELECT col1, IDENTIFIER((SELECT col1)) FROM VALUES(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery(col1)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 37, + "fragment" : "(SELECT col1)" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1', 'col2')) FROM VALUES(1,2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\", \"TIME\"", + "unsupportedType" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 32, + "fragment" : "SELECT 'col1'" + } ] +} + + +-- !query +DROP TABLE t +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + -- !query CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index c90165d31e90d..10d14eebc1d47 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -119,6 +119,14 @@ VALUES(IDENTIFIER(1)); VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))); SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1); +CREATE TABLE t(col1 INT); +SELECT * FROM IDENTIFIER((SELECT 't')); +SELECT * FROM (SELECT IDENTIFIER((SELECT 'col1')) FROM IDENTIFIER((SELECT 't'))); +SELECT IDENTIFIER((SELECT 'col1')) FROM VALUES(1); +SELECT col1, IDENTIFIER((SELECT col1)) FROM VALUES(1); +SELECT IDENTIFIER((SELECT 'col1', 'col2')) FROM VALUES(1,2); +DROP TABLE t; + CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv; CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv; CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1); diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out index ad351074692f5..79f0f8e77c767 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -846,6 +846,137 @@ org.apache.spark.sql.AnalysisException } +-- !query +CREATE TABLE t(col1 INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM IDENTIFIER((SELECT 't')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 37, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT * FROM (SELECT IDENTIFIER((SELECT 'col1')) FROM IDENTIFIER((SELECT 't'))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 78, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1')) FROM VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 33, + "fragment" : "(SELECT 'col1')" + } ] +} + + +-- !query +SELECT col1, IDENTIFIER((SELECT col1)) FROM VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery(col1)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 37, + "fragment" : "(SELECT col1)" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1', 'col2')) FROM VALUES(1,2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\", \"TIME\"", + "unsupportedType" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 32, + "fragment" : "SELECT 'col1'" + } ] +} + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + -- !query CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 4a0f9abe56394..2e2efa2fdc27b 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -846,6 +846,137 @@ org.apache.spark.sql.AnalysisException } +-- !query +CREATE TABLE t(col1 INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM IDENTIFIER((SELECT 't')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 37, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT * FROM (SELECT IDENTIFIER((SELECT 'col1')) FROM IDENTIFIER((SELECT 't'))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 78, + "fragment" : "(SELECT 't')" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1')) FROM VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery()", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 33, + "fragment" : "(SELECT 'col1')" + } ] +} + + +-- !query +SELECT col1, IDENTIFIER((SELECT col1)) FROM VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "scalarsubquery(col1)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 37, + "fragment" : "(SELECT col1)" + } ] +} + + +-- !query +SELECT IDENTIFIER((SELECT 'col1', 'col2')) FROM VALUES(1,2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\", \"TIME\"", + "unsupportedType" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 32, + "fragment" : "SELECT 'col1'" + } ] +} + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + -- !query CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv -- !query schema From defb804475a82ccaa397f71ec83a1a528506b6f0 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 20 Nov 2025 09:23:31 -0800 Subject: [PATCH 177/400] [SPARK-54435][SDP] spark-pipelines init should avoid overwriting existing directory ### What changes were proposed in this pull request? If the name provided to `spark-pipelines init` matches an existing directory, raises an error instead of overwriting the existing directory's contents. ### Why are the changes needed? Help users avoid accidentally overwriting their code. ### Does this PR introduce _any_ user-facing change? Yes, to an unreleased version. ### How was this patch tested? Added unit test for the cases where init is invoked with an existing directory. ### Was this patch authored or co-authored using generative AI tooling? Closes #53140 from sryza/initoverwrite. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 6811eabeb147b0d902dd7dc4fc5b8f29437e54b6) Signed-off-by: Dongjoon Hyun --- python/pyspark/pipelines/init_cli.py | 5 +++++ python/pyspark/pipelines/tests/test_init_cli.py | 15 +++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/python/pyspark/pipelines/init_cli.py b/python/pyspark/pipelines/init_cli.py index f8149b19263f8..bd18c976e6f67 100644 --- a/python/pyspark/pipelines/init_cli.py +++ b/python/pyspark/pipelines/init_cli.py @@ -44,6 +44,11 @@ def example_python_materialized_view() -> DataFrame: def init(name: str) -> None: """Generates a simple pipeline project.""" project_dir = Path.cwd() / name + if project_dir.exists(): + raise FileExistsError( + f"Directory '{name}' already exists. " + "Please choose a different name or remove the existing directory." + ) project_dir.mkdir(parents=True, exist_ok=False) # Create the storage directory diff --git a/python/pyspark/pipelines/tests/test_init_cli.py b/python/pyspark/pipelines/tests/test_init_cli.py index e51bab6a4a691..f88956b647acc 100644 --- a/python/pyspark/pipelines/tests/test_init_cli.py +++ b/python/pyspark/pipelines/tests/test_init_cli.py @@ -72,6 +72,21 @@ def test_init(self): Path("transformations") / "example_sql_materialized_view.sql", ) + def test_init_existing_directory(self): + with tempfile.TemporaryDirectory() as temp_dir: + project_name = "test_project" + with change_dir(Path(temp_dir)): + init(project_name) + + with self.assertRaises(FileExistsError) as context: + init(project_name) + + expected_message = ( + f"Directory '{project_name}' already exists. " + "Please choose a different name or remove the existing directory." + ) + self.assertEqual(str(context.exception), expected_message) + if __name__ == "__main__": try: From ed3847ad917a8419d9f1255f849d86344ef7f142 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 20 Nov 2025 10:59:14 -0800 Subject: [PATCH 178/400] [SPARK-54436][SQL] Fix error formatting for incompatible table metadata checks ### What changes were proposed in this pull request? This PR fixes error formatting for recently added incompatible table metadata checks. ### Why are the changes needed? These changes are needed to avoid unnecessary empty lines. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53115 from aokolnychyi/spark-53924-54157-followup. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun (cherry picked from commit 8cab0745e4042dd99df5c483d1391d2c52b94ec6) Signed-off-by: Dongjoon Hyun --- .../sql/errors/QueryCompilationErrors.scala | 8 +++--- .../DataSourceV2DataFrameSuite.scala | 28 ++++++++----------- 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index f741c5c3975e4..77b775f6c49f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2140,7 +2140,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat errorClass = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", messageParameters = Map( "tableName" -> toSQLId(tableName), - "errors" -> errors.mkString("\n- ", "\n- ", ""))) + "errors" -> errors.mkString("- ", "\n- ", ""))) } def metadataColumnsChangedAfterAnalysis( @@ -2150,7 +2150,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat errorClass = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.METADATA_COLUMNS_MISMATCH", messageParameters = Map( "tableName" -> toSQLId(tableName), - "errors" -> errors.mkString("\n- ", "\n- ", ""))) + "errors" -> errors.mkString("- ", "\n- ", ""))) } def numberOfPartitionsNotAllowedWithUnspecifiedDistributionError(): Throwable = { @@ -4477,7 +4477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "viewName" -> toSQLId(viewName), "tableName" -> toSQLId(tableName), "colType" -> "data", - "errors" -> errors.mkString("\n- ", "\n- ", ""))) + "errors" -> errors.mkString("- ", "\n- ", ""))) } def metadataColumnsChangedAfterViewWithPlanCreation( @@ -4490,6 +4490,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "viewName" -> toSQLId(viewName), "tableName" -> toSQLId(tableName), "colType" -> "metadata", - "errors" -> errors.mkString("\n- ", "\n- ", ""))) + "errors" -> errors.mkString("- ", "\n- ", ""))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 205fa561a5b00..d41af4f1465e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1117,7 +1117,7 @@ class DataSourceV2DataFrameSuite condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "errors" -> "\n- `extra` STRING has been removed")) + "errors" -> "- `extra` STRING has been removed")) } } @@ -1141,8 +1141,7 @@ class DataSourceV2DataFrameSuite parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "errors" -> - """ - |- `new_col1` INT has been added + """- `new_col1` INT has been added |- `new_col2` INT has been added""".stripMargin)) } } @@ -1167,8 +1166,7 @@ class DataSourceV2DataFrameSuite parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "errors" -> - """ - |- `col4` STRING has been removed + """- `col4` STRING has been removed |- `col5` INT has been added""".stripMargin)) } } @@ -1195,9 +1193,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> - """ - |- `extra` INT has been added""".stripMargin)) + "errors" -> "- `extra` INT has been added")) } } @@ -1220,7 +1216,7 @@ class DataSourceV2DataFrameSuite parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "errors" -> - ("\n- `person` type has changed from STRUCT " + + ("- `person` type has changed from STRUCT " + "to STRUCT"))) } } @@ -1270,7 +1266,7 @@ class DataSourceV2DataFrameSuite condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "errors" -> "\n- `extra` STRING has been added")) + "errors" -> "- `extra` STRING has been added")) // DataFrame execution before joins must have pinned used versions // subsequent version refreshes must not be visible in original DataFrames @@ -1400,7 +1396,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "\n- `age` INT has been added")) + "errors" -> "- `age` INT has been added")) } } @@ -1424,7 +1420,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "\n- `age` INT has been removed")) + "errors" -> "- `age` INT has been removed")) } } @@ -1448,7 +1444,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "\n- `data` type has changed from STRING NOT NULL to STRING")) + "errors" -> "- `data` type has changed from STRING NOT NULL to STRING")) } } @@ -1537,7 +1533,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "\n- `age` INT has been added")) + "errors" -> "- `age` INT has been added")) } } @@ -1566,7 +1562,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "\n- `age` INT has been added")) + "errors" -> "- `age` INT has been added")) } } } @@ -1591,7 +1587,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "\n- `name` type has changed from VARCHAR(10) to VARCHAR(20)")) + "errors" -> "- `name` type has changed from VARCHAR(10) to VARCHAR(20)")) } } From c529925af1c57b51c2008e769ab953bb47bf7cd7 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Thu, 20 Nov 2025 13:08:19 -0800 Subject: [PATCH 179/400] [SPARK-54418][SDP] Fix error messages and code formatting ### What changes were proposed in this pull request? Fixes typos and cleans up code formatting (that could've been automated, but done manually this time) https://issues.apache.org/jira/browse/SPARK-54418 ### Why are the changes needed? Cleaner code with a fewer typos ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Built locally. Waiting for the official build to finish once PR's created. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52538 from jaceklaskowski/sdp-typo-hunting-formatting. Authored-by: Jacek Laskowski Signed-off-by: Dongjoon Hyun (cherry picked from commit 4ac02ab225f38429ba01a9ee064f8027c8f94e97) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 2 +- .../apache/spark/deploy/SparkPipelines.scala | 8 +-- python/pyspark/pipelines/api.py | 4 +- python/pyspark/pipelines/cli.py | 21 ++++-- .../pipelines/spark_connect_pipeline.py | 5 +- .../catalyst/plans/logical/v2Commands.scala | 6 +- .../pipelines/DataflowGraphRegistry.scala | 5 +- .../connect/pipelines/PipelinesHandler.scala | 44 ++++++------- .../spark/sql/execution/SparkSqlParser.scala | 2 +- ...pelineDatasetAsSelectParserSuiteBase.scala | 2 +- ...eMaterializedViewAsSelectParserSuite.scala | 2 +- .../graph/CoreDataflowNodeProcessor.scala | 13 ++-- .../sql/pipelines/graph/DataflowGraph.scala | 10 ++- .../graph/DataflowGraphTransformer.scala | 3 +- .../sql/pipelines/graph/DatasetManager.scala | 20 +++--- .../spark/sql/pipelines/graph/Flow.scala | 10 +-- .../sql/pipelines/graph/FlowAnalysis.scala | 64 +++++++++---------- .../sql/pipelines/graph/FlowExecution.scala | 32 ++++------ .../sql/pipelines/graph/FlowPlanner.scala | 7 +- .../graph/GraphRegistrationContext.scala | 33 +++++++--- .../pipelines/graph/PipelineExecution.scala | 4 +- .../graph/SqlGraphRegistrationContext.scala | 18 ++---- .../graph/TriggeredGraphExecution.scala | 10 +-- .../sql/pipelines/graph/ViewHelpers.scala | 2 +- .../spark/sql/pipelines/graph/elements.scala | 2 +- .../sql/pipelines/logging/PipelineEvent.scala | 7 +- .../utils/TestGraphRegistrationContext.scala | 2 +- 27 files changed, 171 insertions(+), 167 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index b190e36a5029b..555f310e512e3 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5248,7 +5248,7 @@ "RUN_EMPTY_PIPELINE" : { "message" : [ "Pipelines are expected to have at least one non-temporary dataset defined (tables, persisted views) but no non-temporary datasets were found in your pipeline.", - "Please verify that you have included the expected source files, and that your source code includes table definitions (e.g., CREATE MATERIALIZED VIEW in SQL code, @sdp.table in python code)." + "Please verify that you have included the expected source files, and that your source code includes table definitions (e.g., CREATE MATERIALIZED VIEW in SQL code, @dp.table in python code)." ], "sqlState" : "42617" }, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkPipelines.scala b/core/src/main/scala/org/apache/spark/deploy/SparkPipelines.scala index 713937cadabfb..ee3bbd88646ff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkPipelines.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkPipelines.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.util +import java.util.{Arrays => JArrays, List => JList} import java.util.Locale import scala.collection.mutable.ArrayBuffer @@ -46,7 +46,7 @@ object SparkPipelines extends Logging { pipelinesCliFile: String, args: Array[String]): Seq[String] = { val (sparkSubmitArgs, pipelinesArgs) = splitArgs(args) - (sparkSubmitArgs ++ Seq(pipelinesCliFile) ++ pipelinesArgs) + sparkSubmitArgs ++ Seq(pipelinesCliFile) ++ pipelinesArgs } /** @@ -59,7 +59,7 @@ object SparkPipelines extends Logging { var remote = "local" new SparkSubmitArgumentsParser() { - parse(util.Arrays.asList(args: _*)) + parse(JArrays.asList(args: _*)) override protected def handle(opt: String, value: String): Boolean = { if (opt == "--remote") { @@ -91,7 +91,7 @@ object SparkPipelines extends Logging { true } - override protected def handleExtraArgs(extra: util.List[String]): Unit = { + override protected def handleExtraArgs(extra: JList[String]): Unit = { pipelinesArgs.appendAll(extra.asScala) } diff --git a/python/pyspark/pipelines/api.py b/python/pyspark/pipelines/api.py index c01e8524eee20..7f8ba4297f3c7 100644 --- a/python/pyspark/pipelines/api.py +++ b/python/pyspark/pipelines/api.py @@ -470,12 +470,12 @@ def create_sink( options: Optional[Dict[str, str]] = None, ) -> None: """ - Creates a sink that can be targeted by streaming flows, providing a generic destination \ + Creates a sink that can be targeted by streaming flows, providing a generic destination for flows to send data external to the pipeline. :param name: The name of the sink. :param format: The format of the sink, e.g. "parquet". - :param options: A dict where the keys are the property names and the values are the \ + :param options: A dict where the keys are the property names and the values are the property values. These properties will be set on the sink. """ if type(name) is not str: diff --git a/python/pyspark/pipelines/cli.py b/python/pyspark/pipelines/cli.py index 3ba0bb58fe946..c06683437fee8 100644 --- a/python/pyspark/pipelines/cli.py +++ b/python/pyspark/pipelines/cli.py @@ -225,8 +225,8 @@ def register_definitions( dataflow_graph_id: str, ) -> None: """Register the graph element definitions in the pipeline spec with the given registry. - - Looks for Python files matching the glob patterns in the spec and imports them. - - Looks for SQL files matching the blob patterns in the spec and registers thems. + - Import Python files matching the glob patterns in the spec. + - Register SQL files matching the glob patterns in the spec. """ path = spec_path.parent with change_dir(path): @@ -356,8 +356,9 @@ def parse_table_list(value: str) -> List[str]: return [table.strip() for table in value.split(",") if table.strip()] -if __name__ == "__main__": - parser = argparse.ArgumentParser(description="Pipeline CLI") +def main() -> None: + """The entry point of spark-pipelines CLI.""" + parser = argparse.ArgumentParser(description="Pipelines CLI") subparsers = parser.add_subparsers(dest="command", required=True) # "run" subcommand @@ -375,7 +376,9 @@ def parse_table_list(value: str) -> List[str]: default=[], ) run_parser.add_argument( - "--full-refresh-all", action="store_true", help="Perform a full graph reset and recompute." + "--full-refresh-all", + action="store_true", + help="Perform a full graph reset and recompute.", ) run_parser.add_argument( "--refresh", @@ -395,7 +398,7 @@ def parse_table_list(value: str) -> List[str]: # "init" subcommand init_parser = subparsers.add_parser( "init", - help="Generates a simple pipeline project, including a spec file and example definitions.", + help="Generate a sample pipeline project, with a spec file and example transformations.", ) init_parser.add_argument( "--name", @@ -424,7 +427,7 @@ def parse_table_list(value: str) -> List[str]: full_refresh=args.full_refresh, full_refresh_all=args.full_refresh_all, refresh=args.refresh, - dry=args.command == "dry-run", + dry=False, ) else: assert args.command == "dry-run" @@ -437,3 +440,7 @@ def parse_table_list(value: str) -> List[str]: ) elif args.command == "init": init(args.name) + + +if __name__ == "__main__": + main() diff --git a/python/pyspark/pipelines/spark_connect_pipeline.py b/python/pyspark/pipelines/spark_connect_pipeline.py index e3c1184cea39d..2fd11d7cf6692 100644 --- a/python/pyspark/pipelines/spark_connect_pipeline.py +++ b/python/pyspark/pipelines/spark_connect_pipeline.py @@ -29,7 +29,7 @@ def create_dataflow_graph( default_database: Optional[str], sql_conf: Optional[Mapping[str, str]], ) -> str: - """Create a dataflow graph in in the Spark Connect server. + """Create a dataflow graph in the Spark Connect server. :returns: The ID of the created dataflow graph. """ @@ -57,7 +57,7 @@ def handle_pipeline_events(iter: Iterator[Dict[str, Any]]) -> None: continue elif "pipeline_event_result" not in result.keys(): raise PySparkValueError( - "Pipeline logs stream handler received an unexpected result: " f"{result}" + f"Pipeline logs stream handler received an unexpected result: {result}" ) else: event = result["pipeline_event_result"].event @@ -76,6 +76,7 @@ def start_run( ) -> Iterator[Dict[str, Any]]: """Start a run of the dataflow graph in the Spark Connect server. + :param spark: SparkSession. :param dataflow_graph_id: The ID of the dataflow graph to start. :param full_refresh: List of datasets to reset and recompute. :param full_refresh_all: Perform a full graph reset and recompute. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 614b73b1547f2..dcce220402449 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -546,7 +546,6 @@ case class CreateTableAsSelect( * The base command representation for a statement that can be part of a Declarative Pipeline to * define a pipeline dataset (MV or ST). */ - trait CreatePipelineDataset extends Command { // The name of the dataset. val name: LogicalPlan @@ -567,7 +566,8 @@ trait CreatePipelineDataset extends Command { /** * An extension of the base command representation that represents a CTAS style CREATE statement. */ -trait CreatePipelineDatasetAsSelect extends BinaryCommand +trait CreatePipelineDatasetAsSelect + extends BinaryCommand with CreatePipelineDataset with CTEInChildren { @@ -2003,7 +2003,7 @@ case class Call( * representation of the matching SQL syntax and cannot be executed. Instead, it is interpreted by * the pipelines submodule during a pipeline execution * - * @param name the name of this flow + * @param name the name of this flow * @param flowOperation the logical plan of the actual transformation this flow should execute * @param comment an optional comment describing this flow */ diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala index e0c7beb43001d..e8114f38ec40c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala @@ -32,13 +32,14 @@ class DataflowGraphRegistry { private val dataflowGraphs = new ConcurrentHashMap[String, GraphRegistrationContext]() - /** Registers a DataflowGraph and generates a unique id to associate with the graph */ + /** + * Registers a GraphRegistrationContext and generates a unique id to associate with the graph + */ def createDataflowGraph( defaultCatalog: String, defaultDatabase: String, defaultSqlConf: Map[String, String]): String = { val graphId = java.util.UUID.randomUUID().toString - // TODO: propagate pipeline catalog and schema from pipeline spec here. dataflowGraphs.put( graphId, new GraphRegistrationContext(defaultCatalog, defaultDatabase, defaultSqlConf)) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala index 4c60e0f70ff4c..62f060014117c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala @@ -49,8 +49,6 @@ private[connect] object PipelinesHandler extends Logging { * Command to be handled * @param responseObserver * The response observer where the response will be sent - * @param sparkSession - * The spark session * @param transformRelationFunc * Function used to convert a relation to a LogicalPlan. This is used when determining the * LogicalPlan that a flow returns. @@ -87,9 +85,7 @@ private[connect] object PipelinesHandler extends Logging { defineOutput(cmd.getDefineOutput, sessionHolder) val identifierBuilder = ResolvedIdentifier.newBuilder() resolvedDataset.catalog.foreach(identifierBuilder.setCatalogName) - resolvedDataset.database.foreach { ns => - identifierBuilder.addNamespace(ns) - } + resolvedDataset.database.foreach(identifierBuilder.addNamespace) identifierBuilder.setTableName(resolvedDataset.identifier) val identifier = identifierBuilder.build() PipelineCommandResult @@ -116,7 +112,7 @@ private[connect] object PipelinesHandler extends Logging { .setDefineFlowResult( PipelineCommandResult.DefineFlowResult .newBuilder() - .setResolvedIdentifier(identifierBuilder) + .setResolvedIdentifier(identifier) .build()) .build() case proto.PipelineCommand.CommandTypeCase.START_RUN => @@ -181,15 +177,21 @@ private[connect] object PipelinesHandler extends Logging { val defaultCatalog = Option .when(cmd.hasDefaultCatalog)(cmd.getDefaultCatalog) .getOrElse { - logInfo(s"No default catalog was supplied. Falling back to the current catalog.") - sessionHolder.session.catalog.currentCatalog() + val currentCatalog = sessionHolder.session.catalog.currentCatalog() + logInfo( + "No default catalog was supplied. " + + s"Falling back to the current catalog: $currentCatalog.") + currentCatalog } val defaultDatabase = Option .when(cmd.hasDefaultDatabase)(cmd.getDefaultDatabase) .getOrElse { - logInfo(s"No default database was supplied. Falling back to the current database.") - sessionHolder.session.catalog.currentDatabase + val currentDatabase = sessionHolder.session.catalog.currentDatabase + logInfo( + "No default database was supplied. " + + s"Falling back to the current database: $currentDatabase.") + currentDatabase } val defaultSqlConf = cmd.getSqlConfMap.asScala.toMap @@ -280,18 +282,15 @@ private[connect] object PipelinesHandler extends Logging { output.getSourceCodeLocation.getFileName), line = Option.when(output.getSourceCodeLocation.hasLineNumber)( output.getSourceCodeLocation.getLineNumber), - objectType = Option(QueryOriginType.View.toString), + objectType = Some(QueryOriginType.View.toString), objectName = Option(viewIdentifier.unquotedString), - language = Option(Python())), + language = Some(Python())), properties = Map.empty, sqlText = None)) viewIdentifier case proto.OutputType.SINK => - val dataflowGraphId = output.getDataflowGraphId - val graphElementRegistry = - sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId) val identifier = GraphIdentifierManager - .parseTableIdentifier(name = output.getOutputName, spark = sessionHolder.session) + .parseTableIdentifier(output.getOutputName, sessionHolder.session) val sinkDetails = output.getSinkDetails graphElementRegistry.registerSink( SinkImpl( @@ -305,7 +304,7 @@ private[connect] object PipelinesHandler extends Logging { output.getSourceCodeLocation.getLineNumber), objectType = Option(QueryOriginType.Sink.toString), objectName = Option(identifier.unquotedString), - language = Option(Python())))) + language = Some(Python())))) identifier case _ => throw new IllegalArgumentException(s"Unknown output type: ${output.getOutputType}") @@ -342,8 +341,7 @@ private[connect] object PipelinesHandler extends Logging { val rawDestinationIdentifier = GraphIdentifierManager .parseTableIdentifier(name = flow.getTargetDatasetName, spark = sessionHolder.session) val flowWritesToView = - graphElementRegistry - .getViews() + graphElementRegistry.getViews .filter(_.isInstanceOf[TemporaryView]) .exists(_.identifier == rawDestinationIdentifier) val flowWritesToSink = @@ -353,7 +351,7 @@ private[connect] object PipelinesHandler extends Logging { // If the flow is created implicitly as part of defining a view or that it writes to a sink, // then we do not qualify the flow identifier and the flow destination. This is because // views and sinks are not permitted to have multipart - val isImplicitFlowForTempView = (isImplicitFlow && flowWritesToView) + val isImplicitFlowForTempView = isImplicitFlow && flowWritesToView val Seq(flowIdentifier, destinationIdentifier) = Seq(rawFlowIdentifier, rawDestinationIdentifier).map { rawIdentifier => if (isImplicitFlowForTempView || flowWritesToSink) { @@ -370,7 +368,7 @@ private[connect] object PipelinesHandler extends Logging { val relationFlowDetails = flow.getRelationFlowDetails graphElementRegistry.registerFlow( - new UnresolvedFlow( + UnresolvedFlow( identifier = flowIdentifier, destinationIdentifier = destinationIdentifier, func = FlowAnalysis.createFlowFunctionFromLogicalPlan( @@ -383,9 +381,9 @@ private[connect] object PipelinesHandler extends Logging { flow.getSourceCodeLocation.getFileName), line = Option.when(flow.getSourceCodeLocation.hasLineNumber)( flow.getSourceCodeLocation.getLineNumber), - objectType = Option(QueryOriginType.Flow.toString), + objectType = Some(QueryOriginType.Flow.toString), objectName = Option(flowIdentifier.unquotedString), - language = Option(Python())))) + language = Some(Python())))) flowIdentifier } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 550c23e3e830c..f8f6e31be1bcc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1385,7 +1385,7 @@ class SparkSqlAstBuilder extends AstBuilder { if (colConstraints.nonEmpty) { throw operationNotAllowed("Pipeline datasets do not currently support column constraints. " + - "Please remove and CHECK, UNIQUE, PK, and FK constraints specified on the pipeline " + + "Please remove any CHECK, UNIQUE, PK, and FK constraints specified on the pipeline " + "dataset.", ctx) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreatePipelineDatasetAsSelectParserSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreatePipelineDatasetAsSelectParserSuiteBase.scala index 9eb82853a3168..4ba4601ca729e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreatePipelineDatasetAsSelectParserSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreatePipelineDatasetAsSelectParserSuiteBase.scala @@ -110,7 +110,7 @@ trait CreatePipelineDatasetAsSelectParserSuiteBase extends CommandSuiteBase { exception = ex, condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> ("Pipeline datasets do not currently support column " + - "constraints. Please remove and CHECK, UNIQUE, PK, and FK constraints specified on the " + + "constraints. Please remove any CHECK, UNIQUE, PK, and FK constraints specified on the " + "pipeline dataset.")), queryContext = ex.getQueryContext.map(toExpectedContext) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateMaterializedViewAsSelectParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateMaterializedViewAsSelectParserSuite.scala index 1185a0e3a8ee1..d8e3a366ff0ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateMaterializedViewAsSelectParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateMaterializedViewAsSelectParserSuite.scala @@ -29,7 +29,7 @@ class CreateMaterializedViewAsSelectParserSuite test("Cannot create materialized view without subquery") { val ex = intercept[ParseException] { - parser.parsePlan(s"CREATE MATERIALIZED VIEW table") + parser.parsePlan(s"CREATE $datasetSqlSyntax table") } assert(ex.getMessage.contains(s"Unable to find query for CREATE $datasetSqlSyntax statement.")) } diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/CoreDataflowNodeProcessor.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/CoreDataflowNodeProcessor.scala index b87c02d562cb8..38fde0bfec4a1 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/CoreDataflowNodeProcessor.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/CoreDataflowNodeProcessor.scala @@ -127,8 +127,7 @@ private class FlowResolver(rawGraph: DataflowGraph) { queryContext = flowToResolve.queryContext, queryOrigin = flowToResolve.origin ) - val result = - flowFunctionResult match { + flowFunctionResult match { case f if f.dataFrame.isSuccess => // Merge confs from any upstream views into confs for this flow. val allFConfs = @@ -198,22 +197,20 @@ private class FlowResolver(rawGraph: DataflowGraph) { ) } } - result } private def convertResolvedToTypedFlow( flow: UnresolvedFlow, funcResult: FlowFunctionResult): ResolvedFlow = { - val typedFlow = flow match { - case f: UnresolvedFlow if f.once => new AppendOnceFlow(flow, funcResult) - case f: UnresolvedFlow if funcResult.dataFrame.get.isStreaming => + flow match { + case _ if flow.once => new AppendOnceFlow(flow, funcResult) + case _ if funcResult.dataFrame.get.isStreaming => // If there's more than 1 flow to this flow's destination, we should not allow it // to be planned with an output mode other than Append, as the other flows will // then get their results overwritten. - val mustBeAppend = rawGraph.flowsTo(f.destinationIdentifier).size > 1 + val mustBeAppend = rawGraph.flowsTo(flow.destinationIdentifier).size > 1 new StreamingFlow(flow, funcResult, mustBeAppend = mustBeAppend) case _: UnresolvedFlow => new CompleteFlow(flow, funcResult) } - typedFlow } } diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraph.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraph.scala index a1ae49b413a10..c9578ddd3b469 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraph.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraph.scala @@ -30,9 +30,13 @@ import org.apache.spark.sql.types.StructType * It manages the relationships between logical flows, tables, sinks, and views, providing * operations for graph traversal, validation, and transformation. */ -case class DataflowGraph(flows: Seq[Flow], tables: Seq[Table], sinks: Seq[Sink], views: Seq[View]) - extends GraphOperations - with GraphValidations { +case class DataflowGraph( + flows: Seq[Flow], + tables: Seq[Table], + sinks: Seq[Sink], + views: Seq[View]) + extends GraphOperations + with GraphValidations { /** Map of [[Output]]s by their identifiers */ lazy val output: Map[TableIdentifier, Output] = mapUnique(sinks ++ tables, "output")(_.identifier) diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala index c80978a5957dd..2523c0ae5502a 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DataflowGraphTransformer.scala @@ -157,8 +157,7 @@ class DataflowGraphTransformer(graph: DataflowGraph) extends AutoCloseable { None } } - if (flowOpt.isDefined) { - val flow = flowOpt.get + flowOpt.foreach { flow => futures.append( executor.submit( () => diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala index e5c87fa542ad1..4affbe4637dba 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala @@ -77,7 +77,7 @@ object DatasetManager extends Logging { context: PipelineUpdateContext ): DataflowGraph = { val (_, refreshTableIdentsSet, fullRefreshTableIdentsSet) = { - DatasetManager.constructFullRefreshSet(resolvedDataflowGraph.tables, context) + constructFullRefreshSet(resolvedDataflowGraph.tables, context) } /** Return all the tables that need to be materialized from the given graph. */ @@ -125,7 +125,6 @@ object DatasetManager extends Logging { } catch { case e: SparkException if e.getCause != null => throw e.getCause } - materializeViews(materializedGraph, context) materializedGraph } @@ -144,7 +143,7 @@ object DatasetManager extends Logging { var publishedViews: Set[TableIdentifier] = Set.empty var failedViews: Set[TableIdentifier] = Set.empty - // To publish a view, it is required that all the input sources must exists in the metastore. + // To publish a view, it is required that all the input sources must exist in the metastore. // Thereby, if a Persisted View target reads another Persisted View source, the source must be // published first. // Here we make sure all the persisted views are published in correct order @@ -226,8 +225,8 @@ object DatasetManager extends Logging { try { // Using the catalog and database from the flow ensures that reads within the view are // directed to the right catalog/database. - catalogManager.setCurrentCatalog(queryContext.currentCatalog.get) - queryContext.currentDatabase.foreach { d => catalogManager.setCurrentNamespace(Array(d))} + queryContext.currentCatalog.foreach(catalogManager.setCurrentCatalog) + queryContext.currentDatabase.map(d => Array(d)).foreach(catalogManager.setCurrentNamespace) command.run(spark) } finally { catalogManager.setCurrentCatalog(currentCatalogName) @@ -248,8 +247,7 @@ object DatasetManager extends Logging { resolvedDataflowGraph: DataflowGraph, table: Table, isFullRefresh: Boolean, - context: PipelineUpdateContext - ): Table = { + context: PipelineUpdateContext): Table = { logInfo(log"Materializing metadata for table ${MDC(LogKeys.TABLE_NAME, table.identifier)}.") val catalogManager = context.spark.sessionState.catalogManager val catalog = (table.identifier.catalog match { @@ -287,8 +285,8 @@ object DatasetManager extends Logging { } // Error if partitioning/clustering doesn't match - if (existingTableOpt.isDefined) { - val existingTransforms = existingTableOpt.get.partitioning().toSeq + existingTableOpt.foreach { existingTable => + val existingTransforms = existingTable.partitioning().toSeq if (existingTransforms != allTransforms) { throw new AnalysisException( errorClass = "CANNOT_UPDATE_PARTITION_COLUMNS", @@ -306,8 +304,8 @@ object DatasetManager extends Logging { } // Alter the table if we need to - if (existingTableOpt.isDefined) { - val existingSchema = v2ColumnsToStructType(existingTableOpt.get.columns()) + existingTableOpt.foreach { existingTable => + val existingSchema = v2ColumnsToStructType(existingTable.columns()) val targetSchema = if (table.isStreamingTable && !isFullRefresh) { SchemaMergingUtils.mergeSchemas(existingSchema, outputSchema) diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/Flow.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/Flow.scala index 91feee936170a..e329308502f0d 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/Flow.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/Flow.scala @@ -149,8 +149,10 @@ trait ResolutionCompletedFlow extends Flow { } /** A [[Flow]] whose flow function has failed to resolve. */ -class ResolutionFailedFlow(val flow: UnresolvedFlow, val funcResult: FlowFunctionResult) - extends ResolutionCompletedFlow { +class ResolutionFailedFlow( + val flow: UnresolvedFlow, + val funcResult: FlowFunctionResult) + extends ResolutionCompletedFlow { assert(!funcResult.resolved) def failure: Seq[Throwable] = funcResult.failure @@ -174,14 +176,14 @@ class StreamingFlow( val flow: UnresolvedFlow, val funcResult: FlowFunctionResult, val mustBeAppend: Boolean = false -) extends ResolvedFlow {} +) extends ResolvedFlow /** A [[Flow]] that declares exactly what data should be in the target table. */ class CompleteFlow( val flow: UnresolvedFlow, val funcResult: FlowFunctionResult, val mustBeAppend: Boolean = false -) extends ResolvedFlow {} +) extends ResolvedFlow /** A [[Flow]] that reads source[s] completely and appends data to the target, just once. */ diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowAnalysis.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowAnalysis.scala index 18ae45c4f3405..1a00a6339c4ba 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowAnalysis.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowAnalysis.scala @@ -41,36 +41,32 @@ object FlowAnalysis { * @return A FlowFunction that attempts to analyze the provided LogicalPlan. */ def createFlowFunctionFromLogicalPlan(plan: LogicalPlan): FlowFunction = { - new FlowFunction { - override def call( - allInputs: Set[TableIdentifier], - availableInputs: Seq[Input], - confs: Map[String, String], - queryContext: QueryContext, - queryOrigin: QueryOrigin - ): FlowFunctionResult = { - val ctx = FlowAnalysisContext( - allInputs = allInputs, - availableInputs = availableInputs, - queryContext = queryContext, - spark = SparkSession.active - ) - val df = try { - confs.foreach { case (k, v) => ctx.setConf(k, v) } - Try(FlowAnalysis.analyze(ctx, plan)) - } finally { - ctx.restoreOriginalConf() - } - FlowFunctionResult( - requestedInputs = ctx.requestedInputs.toSet, - batchInputs = ctx.batchInputs.toSet, - streamingInputs = ctx.streamingInputs.toSet, - usedExternalInputs = ctx.externalInputs.toSet, - dataFrame = df, - sqlConf = confs, - analysisWarnings = ctx.analysisWarnings.toList - ) + (allInputs: Set[TableIdentifier], + availableInputs: Seq[Input], + confs: Map[String, String], + queryContext: QueryContext, + queryOrigin: QueryOrigin) => { + val ctx = FlowAnalysisContext( + allInputs = allInputs, + availableInputs = availableInputs, + queryContext = queryContext, + spark = SparkSession.active + ) + val df = try { + confs.foreach { case (k, v) => ctx.setConf(k, v) } + Try(FlowAnalysis.analyze(ctx, plan)) + } finally { + ctx.restoreOriginalConf() } + FlowFunctionResult( + requestedInputs = ctx.requestedInputs.toSet, + batchInputs = ctx.batchInputs.toSet, + streamingInputs = ctx.streamingInputs.toSet, + usedExternalInputs = ctx.externalInputs.toSet, + dataFrame = df, + sqlConf = confs, + analysisWarnings = ctx.analysisWarnings.toList + ) } } @@ -220,7 +216,7 @@ object FlowAnalysis { ctx.requestedInputs += datasetIdentifier - val i = if (!ctx.allInputs.contains(datasetIdentifier)) { + val input = if (!ctx.allInputs.contains(datasetIdentifier)) { // Dataset not defined in the dataflow graph throw GraphErrors.pipelineLocalDatasetNotDefinedError(datasetIdentifier.unquotedString) } else if (!ctx.availableInput.contains(datasetIdentifier)) { @@ -231,8 +227,8 @@ object FlowAnalysis { ctx.availableInput(datasetIdentifier) } - val inputDF = i.load(readOptions) - i match { + val inputDF = input.load(readOptions) + input match { // If the referenced input is a [[Flow]], because the query plans will be fused // together, we also need to fuse their confs. case f: Flow => f.sqlConf.foreach { case (k, v) => ctx.setConf(k, v) } @@ -267,7 +263,7 @@ object FlowAnalysis { droppedOptions = sro.droppedUserOptions.keys.toSeq ) } - ctx.streamingInputs += ResolvedInput(i, aliasIdentifier) + ctx.streamingInputs += ResolvedInput(input, aliasIdentifier) case _ => if (inputDF.isStreaming && incompatibleViewReadCheck) { throw new AnalysisException( @@ -275,7 +271,7 @@ object FlowAnalysis { Map("datasetIdentifier" -> datasetIdentifier.toString) ) } - ctx.batchInputs += ResolvedInput(i, aliasIdentifier) + ctx.batchInputs += ResolvedInput(input, aliasIdentifier) } Dataset.ofRows( ctx.spark, diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala index 647df79bb940c..13a5621947d57 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala @@ -21,7 +21,6 @@ import java.util.concurrent.ThreadPoolExecutor import java.util.concurrent.atomic.AtomicBoolean import scala.concurrent.{ExecutionContext, Future} -import scala.util.{Failure, Success} import scala.util.control.NonFatal import org.apache.spark.internal.{Logging, LogKeys} @@ -53,7 +52,7 @@ trait FlowExecution { def identifier: TableIdentifier /** - * Returns a user-visible name for the flow. + * Returns the user-visible name of this flow. */ final def displayName: String = identifier.unquotedString @@ -139,10 +138,6 @@ trait FlowExecution { _future = try { Option( executeInternal() - .transform { - case Success(_) => Success(ExecutionResult.FINISHED) - case Failure(e) => Failure(e) - } .map(_ => ExecutionResult.FINISHED) .recover { case _: Throwable if stopped.get() => @@ -196,7 +191,7 @@ trait StreamingFlowExecution extends FlowExecution with Logging { /** Visible for testing */ def getStreamingQuery: StreamingQuery = _streamingQuery.getOrElse( - throw new IllegalStateException(s"StreamingPhysicalFlow has not been started") + throw new IllegalStateException("StreamingPhysicalFlow has not been started") ) /** @@ -230,14 +225,13 @@ class StreamingTableWrite( def startStream(): StreamingQuery = { val data = graph.reanalyzeFlow(flow).df - val dataStreamWriter = data.writeStream + val dataStreamWriter = data + .writeStream .queryName(displayName) .option("checkpointLocation", checkpointPath) .trigger(trigger) .outputMode(OutputMode.Append()) - if (destination.format.isDefined) { - dataStreamWriter.format(destination.format.get) - } + destination.format.foreach(dataStreamWriter.format) dataStreamWriter.toTable(destination.identifier.unquotedString) } } @@ -252,28 +246,25 @@ class BatchTableWrite( val sqlConf: Map[String, String] ) extends FlowExecution { - override def isStreaming: Boolean = false + override final def isStreaming: Boolean = false override def getOrigin: QueryOrigin = flow.origin - def executeInternal(): scala.concurrent.Future[Unit] = + def executeInternal(): Future[Unit] = { SparkSessionUtils.withSqlConf(spark, sqlConf.toList: _*) { updateContext.flowProgressEventLogger.recordRunning(flow = flow) val data = graph.reanalyzeFlow(flow).df Future { val dataFrameWriter = data.write - if (destination.format.isDefined) { - dataFrameWriter.format(destination.format.get) - } + destination.format.foreach(dataFrameWriter.format) // In "append" mode with saveAsTable, partition/cluster columns must be specified in query // because the format and options of the existing table is used, and the table could // have been created with partition columns. - if (destination.clusterCols.isDefined) { - val clusterCols = destination.clusterCols.get + destination.clusterCols.foreach { clusterCols => dataFrameWriter.clusterBy(clusterCols.head, clusterCols.tail: _*) } - if (destination.partitionCols.isDefined) { - dataFrameWriter.partitionBy(destination.partitionCols.get: _*) + destination.partitionCols.foreach { partitionCols => + dataFrameWriter.partitionBy(partitionCols: _*) } dataFrameWriter @@ -281,6 +272,7 @@ class BatchTableWrite( .saveAsTable(destination.identifier.unquotedString) } } + } } /** A `StreamingFlowExecution` that writes a streaming `DataFrame` to a `Sink`. */ diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowPlanner.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowPlanner.scala index 3f3a9e2aaac2c..29e2da4a5e13f 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowPlanner.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowPlanner.scala @@ -31,8 +31,7 @@ import org.apache.spark.sql.streaming.Trigger class FlowPlanner( graph: DataflowGraph, updateContext: PipelineUpdateContext, - triggerFor: Flow => Trigger -) { + triggerFor: Flow => Trigger) { /** * Turns a [[Flow]] into an executable [[FlowExecution]]. @@ -76,8 +75,8 @@ class FlowPlanner( ) case _ => throw new UnsupportedOperationException( - s"Streaming flow ${sf.identifier} cannot write to non-table destination: " + - s"${output.getClass.getSimpleName} (${flow.destinationIdentifier})" + s"Unsupported destination type: ${output.getClass.getSimpleName} for " + + s"streaming flow ${sf.identifier} (${flow.destinationIdentifier})" ) } case _ => diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphRegistrationContext.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphRegistrationContext.scala index 26432fd2960f5..dadda0561b19f 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphRegistrationContext.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphRegistrationContext.scala @@ -50,20 +50,26 @@ class GraphRegistrationContext( sinks += sinkDef } - def getViews(): Seq[View] = { - return views.toSeq + def getViews: Seq[View] = { + views.toSeq } - def getSinks: Seq[Sink] = sinks.toSeq + def getSinks: Seq[Sink] = { + sinks.toSeq + } def registerFlow(flowDef: UnresolvedFlow): Unit = { flows += flowDef.copy(sqlConf = defaultSqlConf ++ flowDef.sqlConf) } + private def isEmpty: Boolean = { + tables.isEmpty && views.collect { case v: PersistedView => + v + }.isEmpty && sinks.isEmpty + } + def toDataflowGraph: DataflowGraph = { - if (tables.isEmpty && views.collect { case v: PersistedView => - v - }.isEmpty && sinks.isEmpty) { + if (isEmpty) { throw new AnalysisException( errorClass = "RUN_EMPTY_PIPELINE", messageParameters = Map.empty) @@ -137,11 +143,22 @@ class GraphRegistrationContext( } } + /** + * Throws an exception if the given flow's identifier is used by multiple flows. + * + * @param flow The flow to check. + * @param datasetType The type of dataset the flow writes to. + * @param flows All flows in the graph. + * @throws AnalysisException If the flow's identifier is used by multiple flows. + */ private def assertFlowIdentifierIsUnique( flow: UnresolvedFlow, flows: Seq[UnresolvedFlow]): Unit = { - flows.groupBy(i => i.identifier).get(flow.identifier).filter(_.size > 1).foreach { - duplicateFlows => + flows + .groupBy(i => i.identifier) + .get(flow.identifier) + .filter(_.size > 1) + .foreach { duplicateFlows => val duplicateFlow = duplicateFlows.filter(_ != flow).head throw new AnalysisException( errorClass = "PIPELINE_DUPLICATE_IDENTIFIERS.FLOW", diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineExecution.scala index 6090b764d543c..d35d701d44e57 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/PipelineExecution.scala @@ -53,7 +53,7 @@ class PipelineExecution(context: PipelineUpdateContext) { val initializedGraph = DatasetManager.materializeDatasets(resolvedGraph, context) // Execute the graph. - graphExecution = Option( + graphExecution = Some( new TriggeredGraphExecution(initializedGraph, context, onCompletion = terminationReason => { context.eventCallback(constructTerminationEvent(terminationReason)) }) @@ -118,7 +118,7 @@ class PipelineExecution(context: PipelineUpdateContext) { } } - /** Waits for the execution to complete. Only used in tests */ + /** Waits for the execution to complete. */ private[sql] def awaitCompletion(): Unit = { graphExecution.foreach(_.awaitCompletion()) } diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala index 5df12be7f4cfe..947629ab68006 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/SqlGraphRegistrationContext.scala @@ -111,17 +111,12 @@ class SqlGraphRegistrationContext( ) def processSqlFile(sqlText: String, sqlFilePath: String, spark: SparkSession): Unit = { - // Create a registration context for this SQL registration request - val sqlGraphElementRegistrationContext = new SqlGraphRegistrationContext( - graphRegistrationContext - ) - splitSqlFileIntoQueries( spark = spark, sqlFileText = sqlText, sqlFilePath = sqlFilePath ).foreach { case SqlQueryPlanWithOrigin(logicalPlan, queryOrigin) => - sqlGraphElementRegistrationContext.processSqlQuery(logicalPlan, queryOrigin) + processSqlQuery(logicalPlan, queryOrigin) } } @@ -416,7 +411,7 @@ class SqlGraphRegistrationContext( ) .identifier - val (flowTargetDatasetIdentifier, flowQueryLogicalPlan, isOnce) = cf.flowOperation match { + val (flowTargetDatasetIdentifier, flowQueryLogicalPlan) = cf.flowOperation match { case i: InsertIntoStatement => validateInsertIntoFlow(i, queryOrigin) val flowTargetDatasetName = i.table match { @@ -435,7 +430,7 @@ class SqlGraphRegistrationContext( currentDatabase = context.getCurrentDatabaseOpt ) .identifier - (qualifiedFlowTargetDatasetName, i.query, false) + (qualifiedFlowTargetDatasetName, i.query) case _ => throw SqlGraphElementRegistrationException( msg = "Unable flow type. Only INSERT INTO flows are supported.", @@ -457,7 +452,7 @@ class SqlGraphRegistrationContext( destinationIdentifier = qualifiedDestinationIdentifier, func = FlowAnalysis.createFlowFunctionFromLogicalPlan(flowQueryLogicalPlan), sqlConf = context.getSqlConf, - once = isOnce, + once = false, queryContext = QueryContext( currentCatalog = context.getCurrentCatalogOpt, currentDatabase = context.getCurrentDatabaseOpt @@ -506,11 +501,10 @@ class SqlGraphRegistrationContext( private object SetCommandHandler { def handle(setCommand: SetCommand): Unit = { - val sqlConfKvPair = setCommand.kv.getOrElse( + val (sqlConfKey, valueOpt) = setCommand.kv.getOrElse( throw new RuntimeException("Invalid SET command without key-value pair") ) - val sqlConfKey = sqlConfKvPair._1 - val sqlConfValue = sqlConfKvPair._2.getOrElse( + val sqlConfValue = valueOpt.getOrElse( throw new RuntimeException("Invalid SET command without value") ) context.setSqlConf(sqlConfKey, sqlConfValue) diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala index c57f125aed331..02871e64aa761 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import scala.util.Try import scala.util.control.NonFatal -import org.apache.spark.internal.{LogKeys} +import org.apache.spark.internal.LogKeys import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.pipelines.graph.TriggeredGraphExecution._ import org.apache.spark.sql.pipelines.util.ExponentialBackoffStrategy @@ -33,7 +33,7 @@ import org.apache.spark.sql.streaming.Trigger import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** - * Executes all of the flows in the given graph in topological order. Each flow processes + * Executes all the flows in the given graph in topological order. Each flow processes * all available data before downstream flows are triggered. * * @param graphForExecution the graph to execute. @@ -124,7 +124,7 @@ class TriggeredGraphExecution( } ) thread.start() - topologicalExecutionThread = Option(thread) + topologicalExecutionThread = Some(thread) } /** Used to control how many flows are executing at once. */ @@ -211,7 +211,7 @@ class TriggeredGraphExecution( } } - // collect flow that are ready to start + // collect flows that are ready to start val flowsToStart = mutable.ArrayBuffer[ResolvedFlow]() while (runnableFlows.nonEmpty && concurrencyLimit.tryAcquire()) { val flowIdentifier = runnableFlows.head @@ -237,8 +237,8 @@ class TriggeredGraphExecution( concurrencyLimit.release() } else { env.flowProgressEventLogger.recordSkipped(flow) - concurrencyLimit.release() pipelineState.put(flowIdentifier, StreamState.SKIPPED) + concurrencyLimit.release() } } } catch { diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/ViewHelpers.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/ViewHelpers.scala index 9edd0361cb0f9..6a2bbb99d4d7f 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/ViewHelpers.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/ViewHelpers.scala @@ -25,7 +25,7 @@ object ViewHelpers { def persistedViewIdentifierToFlow(graph: DataflowGraph): Map[TableIdentifier, ResolvedFlow] = { graph.persistedViews.map { v => require( - graph.flowsTo.get(v.identifier).isDefined, + graph.flowsTo.contains(v.identifier), s"No flows to view ${v.identifier} were found" ) val flowsToView = graph.resolvedFlowsTo(v.identifier) diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala index c762174e67251..ce3a63de6a333 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala @@ -268,4 +268,4 @@ case class SinkImpl( format: String, options: Map[String, String], origin: QueryOrigin - ) extends Sink {} +) extends Sink {} diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/PipelineEvent.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/PipelineEvent.scala index 0a10ebaa69f7a..3f5ac8f7cb21e 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/PipelineEvent.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/logging/PipelineEvent.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.pipelines.graph.QueryOrigin * @param timestamp The time of the event * @param origin Where the event originated from * @param level Security level of the event - * @param message A user friendly description of the event + * @param message A user-friendly description of the event * @param details The details of the event * @param error An error that occurred during the event */ @@ -38,8 +38,7 @@ case class PipelineEvent( level: EventLevel, message: String, details: EventDetails, - error: Option[Throwable] -) { + error: Option[Throwable]) { /** Combines the message and error (if any) into a single string */ def messageWithError: String = { if (error.nonEmpty) { @@ -49,7 +48,7 @@ case class PipelineEvent( Option(throwable.getCause).map(getExceptionMessages).getOrElse(Nil) } val errorMessages = getExceptionMessages(error.get) - s"""${message} + s"""$message |Error: ${errorMessages.mkString("\n")}""".stripMargin } else { message diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala index d88432d68ca3c..e7c0956385135 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala @@ -308,7 +308,7 @@ class TestGraphRegistrationContext( val rawDestinationIdentifier = GraphIdentifierManager.parseTableIdentifier(destinationName, spark) - val flowWritesToView = getViews() + val flowWritesToView = getViews .filter(_.isInstanceOf[TemporaryView]) .exists(_.identifier == rawDestinationIdentifier) val flowWritesToSink = getSinks From 7575478838c7457c80fe159569783c5ff77a8718 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 20 Nov 2025 14:45:27 -0800 Subject: [PATCH 180/400] [SPARK-51771][SQL][FOLLOWUP] Rename currentVersion to version in DSv2 Table ### What changes were proposed in this pull request? This PR renames `currentVersion` to `version` in DSv2 `Table`. This method is supposed to be a simple getter and must not trigger a refresh of the underlying table state. ### Why are the changes needed? These changes are needed to avoid ambiguity in the about to be released API in 4.1. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53118 from aokolnychyi/spark-51771-followup. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun (cherry picked from commit 6578b9b96e41bd15431d7ea7449683bb8d888bcd) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/connector/catalog/Table.java | 8 +++++--- .../datasources/v2/DataSourceV2Relation.scala | 2 +- .../datasources/v2/V2TableRefreshUtil.scala | 2 +- .../spark/sql/connector/catalog/CatalogSuite.scala | 12 ++++++------ .../sql/connector/catalog/InMemoryBaseTable.scala | 12 ++++++------ .../spark/sql/connector/catalog/InMemoryTable.scala | 10 +++++----- .../sql/connector/catalog/InMemoryTableCatalog.scala | 8 ++++---- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../execution/command/v2/CheckConstraintSuite.scala | 8 ++++---- 9 files changed, 33 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index f85b5e3569298..a298520760bc0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -104,8 +104,10 @@ default Map properties() { default Constraint[] constraints() { return new Constraint[0]; } /** - * Returns the current table version if implementation supports versioning. - * If the table is not versioned, null can be returned here. + * Returns the version of this table if versioning is supported, null otherwise. + *

+ * This method must not trigger a refresh of the table metadata. It should return + * the version that corresponds to the current state of this table instance. */ - default String currentVersion() { return null; } + default String version() { return null; } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 4664c5c6a70c6..fda415d1ab29d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -134,7 +134,7 @@ case class DataSourceV2Relation( def autoSchemaEvolution(): Boolean = table.capabilities().contains(TableCapability.AUTOMATIC_SCHEMA_EVOLUTION) - def isVersioned: Boolean = table.currentVersion != null + def isVersioned: Boolean = table.version != null } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala index e98b80b6a5a0d..0852043cb8228 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala @@ -42,7 +42,7 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { case r @ ExtractV2CatalogAndIdentifier(catalog, ident) if r.isVersioned && r.timeTravelSpec.isEmpty => val tableName = V2TableUtil.toQualifiedName(catalog, ident) - val version = r.table.currentVersion + val version = r.table.version logDebug(s"Pinning table version for $tableName to $version") r.copy(timeTravelSpec = Some(AsOfVersion(version))) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala index 4798623417b1f..b4a4c6f46cda4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala @@ -1357,24 +1357,24 @@ class CatalogSuite extends SparkFunSuite { intercept[NoSuchFunctionException](catalog.loadFunction(Identifier.of(Array("ns1"), "func"))) } - test("currentVersion") { + test("version") { val catalog = newCatalog() val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) .asInstanceOf[InMemoryTable] - assert(table.currentVersion() == "0") + assert(table.version() == "0") table.withData(Array( BufferedRows("3", table.columns()).withRow(InternalRow(0, "abc", "3")), BufferedRows("4", table.columns()).withRow(InternalRow(1, "def", "4")))) - assert(table.currentVersion() == "1") + assert(table.version() == "1") table.truncateTable() - assert(catalog.loadTable(testIdent).currentVersion() == "2") + assert(catalog.loadTable(testIdent).version() == "2") catalog.alterTable(testIdent, TableChange.setProperty("prop-1", "1")) - assert(catalog.loadTable(testIdent).currentVersion() == "3") + assert(catalog.loadTable(testIdent).version() == "3") catalog.alterTable(testIdent, TableChange.addConstraint(constraints.apply(0), "3")) - assert(catalog.loadTable(testIdent).currentVersion() == "4") + assert(catalog.loadTable(testIdent).version() == "4") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala index d66ba5a23cc84..18fe80c2e924e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala @@ -66,7 +66,7 @@ abstract class InMemoryBaseTable( extends Table with SupportsRead with SupportsWrite with SupportsMetadataColumns { // Tracks the current version number of the table. - protected var currentTableVersion: Int = 0 + protected var tableVersion: Int = 0 // Stores the table version validated during the last `ALTER TABLE ... ADD CONSTRAINT` operation. private var validatedTableVersion: String = null @@ -75,14 +75,14 @@ abstract class InMemoryBaseTable( override def columns(): Array[Column] = tableColumns - override def currentVersion(): String = currentTableVersion.toString + override def version(): String = tableVersion.toString - def setCurrentVersion(version: String): Unit = { - currentTableVersion = version.toInt + def setVersion(version: String): Unit = { + tableVersion = version.toInt } - def increaseCurrentVersion(): Unit = { - currentTableVersion += 1 + def increaseVersion(): Unit = { + tableVersion += 1 } def validatedVersion(): String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala index beac7751bf775..3bea136b34d46 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala @@ -69,7 +69,7 @@ class InMemoryTable( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper dataMap --= InMemoryTable .filtersToKeys(dataMap.keys, partCols.map(_.toSeq.quoted).toImmutableArraySeq, filters) - increaseCurrentVersion() + increaseVersion() } override def withData(data: Array[BufferedRows]): InMemoryTable = { @@ -109,7 +109,7 @@ class InMemoryTable( row.getInt(0) == InMemoryTable.uncommittableValue()))) { throw new IllegalArgumentException(s"Test only mock write failure") } - increaseCurrentVersion() + increaseVersion() this } } @@ -155,7 +155,7 @@ class InMemoryTable( copiedTable.commits ++= commits.map(_.copy()) - copiedTable.setCurrentVersion(currentVersion()) + copiedTable.setVersion(version()) if (validatedVersion() != null) { copiedTable.setValidatedVersion(validatedVersion()) } @@ -165,12 +165,12 @@ class InMemoryTable( override def equals(other: Any): Boolean = other match { case that: InMemoryTable => - this.id == that.id && this.currentVersion() == that.currentVersion() + this.id == that.id && this.version() == that.version() case _ => false } override def hashCode(): Int = { - Objects.hash(id, currentVersion()) + Objects.hash(id, version()) } class InMemoryWriterBuilderWithOverWrite(override val info: LogicalWriteInfo) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index 2156dba619eca..5ea33a1764aae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -179,8 +179,8 @@ class BasicInMemoryTableCatalog extends TableCatalog { throw new IllegalArgumentException(s"Cannot drop all fields") } - table.increaseCurrentVersion() - val currentVersion = table.currentVersion() + table.increaseVersion() + val currentVersion = table.version() val newTable = new InMemoryTable( name = table.name, columns = CatalogV2Util.structTypeToV2Columns(schema), @@ -189,7 +189,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { constraints = constraints, id = table.id) .alterTableWithData(table.data, schema) - newTable.setCurrentVersion(currentVersion) + newTable.setVersion(currentVersion) changes.foreach { case a: TableChange.AddConstraint => newTable.setValidatedVersion(a.validatedTableVersion()) @@ -209,7 +209,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { Option(tables.remove(oldIdent)) match { case Some(table: InMemoryBaseTable) => - table.increaseCurrentVersion() + table.increaseVersion() tables.put(newIdent, table) case _ => throw new NoSuchTableException(oldIdent.asMultipartIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 9c624d951a76a..81bc1990404a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -571,7 +571,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val condition = a.checkConstraint.condition val change = TableChange.addConstraint( check.toV2Constraint, - d.relation.table.currentVersion) + d.relation.table.version) ResolveTableConstraints.validateCatalogForTableChange(Seq(change), catalog, ident) AddCheckConstraintExec(catalog, ident, change, condition, planLater(a.child)) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index 1d14d710744a4..44c5b1ad28745 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -207,7 +207,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma sql(s"INSERT INTO $t VALUES (1, 'a'), (null, 'b')") sql(s"ALTER TABLE $t ADD CONSTRAINT c1 CHECK (id > 0) $characteristic") val table = loadTable(nonPartitionCatalog, "ns", "tbl") - assert(table.currentVersion() == "2") + assert(table.version() == "2") assert(table.validatedVersion() == "1") val constraint = getCheckConstraint(table) assert(constraint.name() == "c1") @@ -254,7 +254,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma // Add a valid check constraint sql(s"ALTER TABLE $t ADD CONSTRAINT valid_positive_num CHECK (s.num >= -1)") val table = loadTable(nonPartitionCatalog, "ns", "tbl") - assert(table.currentVersion() == "2") + assert(table.version() == "2") assert(table.validatedVersion() == "1") val constraint = getCheckConstraint(table) assert(constraint.name() == "valid_positive_num") @@ -284,7 +284,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma // Add a valid check constraint sql(s"ALTER TABLE $t ADD CONSTRAINT valid_map_val CHECK (m['a'] >= -1)") val table = loadTable(nonPartitionCatalog, "ns", "tbl") - assert(table.currentVersion() == "2") + assert(table.version() == "2") assert(table.validatedVersion() == "1") val constraint = getCheckConstraint(table) assert(constraint.name() == "valid_map_val") @@ -312,7 +312,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma // Add a valid check constraint sql(s"ALTER TABLE $t ADD CONSTRAINT valid_array CHECK (a[1] >= -2)") val table = loadTable(nonPartitionCatalog, "ns", "tbl") - assert(table.currentVersion() == "2") + assert(table.version() == "2") assert(table.validatedVersion() == "1") val constraint = getCheckConstraint(table) assert(constraint.name() == "valid_array") From 8fd8416f0a930fb1ab77700f4c5cccb2351bae8e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 20 Nov 2025 16:13:07 -0800 Subject: [PATCH 181/400] [SPARK-54440][SDP] Give default pipeline spec file more idiomatic name, `spark-pipeline.yml` ### What changes were proposed in this pull request? Changes the default pipelines configuration file from pipeline.yml to spark-pipeline.yml ### Why are the changes needed? In the current implementation of Declarative Pipelines, the default name for the pipeline configuration YML file is "pipeline.yml". This is inconsistent with other user-provided Spark configuration file names: - spark-env.sh - spark-defaults.conf Changing it to spark-pipeline.yml would be more consistent. ### Does this PR introduce _any_ user-facing change? Not to anything released. ### How was this patch tested? Covered by unit tests. Updated tests. ### Was this patch authored or co-authored using generative AI tooling? Closes #53144 from sryza/spark-pipeline.yml. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit d3346eb6dba3f36f0b91b6cc4d8f05c90354171e) Signed-off-by: Dongjoon Hyun --- .../spark/deploy/SparkPipelinesSuite.scala | 8 +++---- ...declarative-pipelines-programming-guide.md | 4 ++-- python/pyspark/errors/error-conditions.json | 2 +- python/pyspark/pipelines/cli.py | 2 +- python/pyspark/pipelines/init_cli.py | 2 +- python/pyspark/pipelines/tests/test_cli.py | 22 +++++++++---------- 6 files changed, 20 insertions(+), 20 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkPipelinesSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkPipelinesSuite.scala index a97aabfd5a371..60e279ba2ddc5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkPipelinesSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkPipelinesSuite.scala @@ -58,7 +58,7 @@ class SparkPipelinesSuite extends SparkSubmitTestUtils with BeforeAndAfterEach { val args = Array( "run", "--spec", - "pipeline.yml" + "spark-pipeline.yml" ) assert( SparkPipelines.constructSparkSubmitArgs( @@ -71,7 +71,7 @@ class SparkPipelinesSuite extends SparkSubmitTestUtils with BeforeAndAfterEach { "abc/python/pyspark/pipelines/cli.py", "run", "--spec", - "pipeline.yml" + "spark-pipeline.yml" ) ) } @@ -83,7 +83,7 @@ class SparkPipelinesSuite extends SparkSubmitTestUtils with BeforeAndAfterEach { "run", "--supervise", "--spec", - "pipeline.yml", + "spark-pipeline.yml", "--conf", "spark.conf2=3" ) @@ -101,7 +101,7 @@ class SparkPipelinesSuite extends SparkSubmitTestUtils with BeforeAndAfterEach { "abc/python/pyspark/pipelines/cli.py", "run", "--spec", - "pipeline.yml" + "spark-pipeline.yml" ) ) } diff --git a/docs/declarative-pipelines-programming-guide.md b/docs/declarative-pipelines-programming-guide.md index 3932d472cf659..0ca04c644f1ba 100644 --- a/docs/declarative-pipelines-programming-guide.md +++ b/docs/declarative-pipelines-programming-guide.md @@ -96,7 +96,7 @@ configuration: spark.sql.shuffle.partitions: "1000" ``` -It's conventional to name pipeline spec files `pipeline.yml`. +It's conventional to name pipeline spec files `spark-pipeline.yml`. The `spark-pipelines init` command, described below, makes it easy to generate a pipeline project with default configuration and directory structure. @@ -113,7 +113,7 @@ The `spark-pipelines` command line interface (CLI) is the primary way to execute ### `spark-pipelines run` -`spark-pipelines run` launches an execution of a pipeline and monitors its progress until it completes. The `--spec` parameter allows selecting the pipeline spec file. If not provided, the CLI will look in the current directory and parent directories for a file named `pipeline.yml` or `pipeline.yaml`. +`spark-pipelines run` launches an execution of a pipeline and monitors its progress until it completes. The `--spec` parameter allows selecting the pipeline spec file. If not provided, the CLI will look in the current directory and parent directories for a file named `spark-pipeline.yml` or `spark-pipeline.yaml`. ### `spark-pipelines dry-run` diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index 51bbdd8625164..295b372cade5c 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -908,7 +908,7 @@ }, "PIPELINE_SPEC_FILE_NOT_FOUND": { "message": [ - "No pipeline.yaml or pipeline.yml file provided in arguments or found in directory `` or readable ancestor directories." + "No spark-pipeline.yaml or spark-pipeline.yml file provided in arguments or found in directory `` or readable ancestor directories." ] }, "PIPELINE_SPEC_INVALID_GLOB_PATTERN": { diff --git a/python/pyspark/pipelines/cli.py b/python/pyspark/pipelines/cli.py index c06683437fee8..3e1c532897245 100644 --- a/python/pyspark/pipelines/cli.py +++ b/python/pyspark/pipelines/cli.py @@ -51,7 +51,7 @@ from pyspark.pipelines.add_pipeline_analysis_context import add_pipeline_analysis_context -PIPELINE_SPEC_FILE_NAMES = ["pipeline.yaml", "pipeline.yml"] +PIPELINE_SPEC_FILE_NAMES = ["spark-pipeline.yaml", "spark-pipeline.yml"] @dataclass(frozen=True) diff --git a/python/pyspark/pipelines/init_cli.py b/python/pyspark/pipelines/init_cli.py index bd18c976e6f67..a1dbdfd9d5586 100644 --- a/python/pyspark/pipelines/init_cli.py +++ b/python/pyspark/pipelines/init_cli.py @@ -59,7 +59,7 @@ def init(name: str) -> None: storage_path = f"file://{storage_dir.resolve()}" # Write the spec file to the project directory - spec_file = project_dir / "pipeline.yml" + spec_file = project_dir / "spark-pipeline.yml" with open(spec_file, "w") as f: spec_content = SPEC.replace("{{ name }}", name).replace("{{ storage_root }}", storage_path) f.write(spec_content) diff --git a/python/pyspark/pipelines/tests/test_cli.py b/python/pyspark/pipelines/tests/test_cli.py index ff3022fa29663..f810ab099b7be 100644 --- a/python/pyspark/pipelines/tests/test_cli.py +++ b/python/pyspark/pipelines/tests/test_cli.py @@ -191,7 +191,7 @@ def test_unpack_pipeline_spec_bad_configuration(self): def test_find_pipeline_spec_in_current_directory(self): with tempfile.TemporaryDirectory() as temp_dir: - spec_path = Path(temp_dir) / "pipeline.yaml" + spec_path = Path(temp_dir) / "spark-pipeline.yaml" with spec_path.open("w") as f: f.write( """ @@ -208,7 +208,7 @@ def test_find_pipeline_spec_in_current_directory(self): def test_find_pipeline_spec_in_current_directory_yml(self): with tempfile.TemporaryDirectory() as temp_dir: - spec_path = Path(temp_dir) / "pipeline.yml" + spec_path = Path(temp_dir) / "spark-pipeline.yml" with spec_path.open("w") as f: f.write( """ @@ -225,10 +225,10 @@ def test_find_pipeline_spec_in_current_directory_yml(self): def test_find_pipeline_spec_in_current_directory_yml_and_yaml(self): with tempfile.TemporaryDirectory() as temp_dir: - with (Path(temp_dir) / "pipeline.yml").open("w") as f: + with (Path(temp_dir) / "spark-pipeline.yml").open("w") as f: f.write("") - with (Path(temp_dir) / "pipeline.yaml").open("w") as f: + with (Path(temp_dir) / "spark-pipeline.yaml").open("w") as f: f.write("") with self.assertRaises(PySparkException) as context: @@ -241,7 +241,7 @@ def test_find_pipeline_spec_in_parent_directory(self): parent_dir = Path(temp_dir) child_dir = Path(temp_dir) / "child" child_dir.mkdir() - spec_path = parent_dir / "pipeline.yaml" + spec_path = parent_dir / "spark-pipeline.yaml" with spec_path.open("w") as f: f.write( """ @@ -296,7 +296,7 @@ def mv2(): registry = LocalGraphElementRegistry() register_definitions( - outer_dir / "pipeline.yaml", registry, spec, self.spark, "test_graph_id" + outer_dir / "spark-pipeline.yaml", registry, spec, self.spark, "test_graph_id" ) self.assertEqual(len(registry.outputs), 1) self.assertEqual(registry.outputs[0].name, "mv1") @@ -319,7 +319,7 @@ def test_register_definitions_file_raises_error(self): registry = LocalGraphElementRegistry() with self.assertRaises(RuntimeError) as context: register_definitions( - outer_dir / "pipeline.yml", registry, spec, self.spark, "test_graph_id" + outer_dir / "spark-pipeline.yml", registry, spec, self.spark, "test_graph_id" ) self.assertIn("This is a test exception", str(context.exception)) @@ -377,7 +377,7 @@ def test_python_import_current_directory(self): registry = LocalGraphElementRegistry() with change_dir(inner_dir2): register_definitions( - inner_dir1 / "pipeline.yaml", + inner_dir1 / "spark-pipeline.yaml", registry, PipelineSpec( name="test_pipeline", @@ -394,7 +394,7 @@ def test_python_import_current_directory(self): def test_full_refresh_all_conflicts_with_full_refresh(self): with tempfile.TemporaryDirectory() as temp_dir: # Create a minimal pipeline spec - spec_path = Path(temp_dir) / "pipeline.yaml" + spec_path = Path(temp_dir) / "spark-pipeline.yaml" with spec_path.open("w") as f: f.write('{"name": "test_pipeline"}') @@ -418,7 +418,7 @@ def test_full_refresh_all_conflicts_with_full_refresh(self): def test_full_refresh_all_conflicts_with_refresh(self): with tempfile.TemporaryDirectory() as temp_dir: # Create a minimal pipeline spec - spec_path = Path(temp_dir) / "pipeline.yaml" + spec_path = Path(temp_dir) / "spark-pipeline.yaml" with spec_path.open("w") as f: f.write('{"name": "test_pipeline"}') @@ -443,7 +443,7 @@ def test_full_refresh_all_conflicts_with_refresh(self): def test_full_refresh_all_conflicts_with_both(self): with tempfile.TemporaryDirectory() as temp_dir: # Create a minimal pipeline spec - spec_path = Path(temp_dir) / "pipeline.yaml" + spec_path = Path(temp_dir) / "spark-pipeline.yaml" with spec_path.open("w") as f: f.write('{"name": "test_pipeline"}') From 4c77b7c911fa7408d96fea31cc1a86ae679b1365 Mon Sep 17 00:00:00 2001 From: Jessie Luo Date: Fri, 21 Nov 2025 06:52:42 -0800 Subject: [PATCH 182/400] [SPARK-54324] Add test for client-user-context-extensions ### What changes were proposed in this pull request? - Add a new unit test that validates thread-local and global user context extension behavior - Add AnalyzePlan support ### Why are the changes needed? The previous test suite doesn't include user context extensions validation. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `python/run-tests --testnames pyspark.sql.tests.connect.client.test_client` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53050 from cookiedough77/jessie.luo/user-context-extension-core. Lead-authored-by: Jessie Luo Co-authored-by: cookiedough77 <160181641+cookiedough77@users.noreply.github.com> Signed-off-by: Sandy Ryza (cherry picked from commit dc3964a52f3b251890dcf921af038f3645086a82) Signed-off-by: Sandy Ryza --- .../sql/tests/connect/client/test_client.py | 104 ++++++++++++++++++ 1 file changed, 104 insertions(+) diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index c189f996cbe43..c876fca241692 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -26,6 +26,7 @@ if should_test_connect: import grpc import google.protobuf.any_pb2 as any_pb2 + import google.protobuf.wrappers_pb2 as wrappers_pb2 from google.rpc import status_pb2 from google.rpc.error_details_pb2 import ErrorInfo import pandas as pd @@ -136,9 +137,11 @@ class MockService: def __init__(self, session_id: str): self._session_id = session_id self.req = None + self.client_user_context_extensions = [] def ExecutePlan(self, req: proto.ExecutePlanRequest, metadata): self.req = req + self.client_user_context_extensions = list(req.user_context.extensions) resp = proto.ExecutePlanResponse() resp.session_id = self._session_id resp.operation_id = req.operation_id @@ -159,12 +162,14 @@ def ExecutePlan(self, req: proto.ExecutePlanRequest, metadata): def Interrupt(self, req: proto.InterruptRequest, metadata): self.req = req + self.client_user_context_extensions = list(req.user_context.extensions) resp = proto.InterruptResponse() resp.session_id = self._session_id return resp def Config(self, req: proto.ConfigRequest, metadata): self.req = req + self.client_user_context_extensions = list(req.user_context.extensions) resp = proto.ConfigResponse() resp.session_id = self._session_id if req.operation.HasField("get"): @@ -177,6 +182,15 @@ def Config(self, req: proto.ConfigRequest, metadata): pair.value = req.operation.get_with_default.pairs[0].value or "true" return resp + def AnalyzePlan(self, req: proto.AnalyzePlanRequest, metadata): + self.req = req + self.client_user_context_extensions = list(req.user_context.extensions) + resp = proto.AnalyzePlanResponse() + resp.session_id = self._session_id + # Return a minimal response with a semantic hash + resp.semantic_hash.result = 12345 + return resp + # The _cleanup_ml_cache invocation will hang in this test (no valid spark cluster) # and it blocks the test process exiting because it is registered as the atexit handler # in `SparkConnectClient` constructor. To bypass the issue, patch the method in the test. @@ -229,6 +243,96 @@ def userId(self) -> Optional[str]: self.assertEqual(client._user_id, "abc") + def test_user_context_extension(self): + client = SparkConnectClient("sc://foo/", use_reattachable_execute=False) + mock = MockService(client._session_id) + client._stub = mock + + try: + exlocal = any_pb2.Any() + exlocal.Pack(wrappers_pb2.StringValue(value="abc")) + exlocal2 = any_pb2.Any() + exlocal2.Pack(wrappers_pb2.StringValue(value="def")) + exglobal = any_pb2.Any() + exglobal.Pack(wrappers_pb2.StringValue(value="ghi")) + exglobal2 = any_pb2.Any() + exglobal2.Pack(wrappers_pb2.StringValue(value="jkl")) + + exlocal_id = client.add_threadlocal_user_context_extension(exlocal) + exglobal_id = client.add_global_user_context_extension(exglobal) + + mock.client_user_context_extensions = [] + command = proto.Command() + client.execute_command(command) + self.assertTrue(exlocal in mock.client_user_context_extensions) + self.assertTrue(exglobal in mock.client_user_context_extensions) + self.assertFalse(exlocal2 in mock.client_user_context_extensions) + self.assertFalse(exglobal2 in mock.client_user_context_extensions) + + client.add_threadlocal_user_context_extension(exlocal2) + + mock.client_user_context_extensions = [] + plan = proto.Plan() + client.semantic_hash(plan) # use semantic_hash to test analyze + self.assertTrue(exlocal in mock.client_user_context_extensions) + self.assertTrue(exglobal in mock.client_user_context_extensions) + self.assertTrue(exlocal2 in mock.client_user_context_extensions) + self.assertFalse(exglobal2 in mock.client_user_context_extensions) + + client.add_global_user_context_extension(exglobal2) + + mock.client_user_context_extensions = [] + client.interrupt_all() + self.assertTrue(exlocal in mock.client_user_context_extensions) + self.assertTrue(exglobal in mock.client_user_context_extensions) + self.assertTrue(exlocal2 in mock.client_user_context_extensions) + self.assertTrue(exglobal2 in mock.client_user_context_extensions) + + client.remove_user_context_extension(exlocal_id) + + mock.client_user_context_extensions = [] + client.get_configs("foo", "bar") + self.assertFalse(exlocal in mock.client_user_context_extensions) + self.assertTrue(exglobal in mock.client_user_context_extensions) + self.assertTrue(exlocal2 in mock.client_user_context_extensions) + self.assertTrue(exglobal2 in mock.client_user_context_extensions) + + client.remove_user_context_extension(exglobal_id) + + mock.client_user_context_extensions = [] + command = proto.Command() + client.execute_command(command) + self.assertFalse(exlocal in mock.client_user_context_extensions) + self.assertFalse(exglobal in mock.client_user_context_extensions) + self.assertTrue(exlocal2 in mock.client_user_context_extensions) + self.assertTrue(exglobal2 in mock.client_user_context_extensions) + + client.clear_user_context_extensions() + + mock.client_user_context_extensions = [] + plan = proto.Plan() + client.semantic_hash(plan) # use semantic_hash to test analyze + self.assertFalse(exlocal in mock.client_user_context_extensions) + self.assertFalse(exglobal in mock.client_user_context_extensions) + self.assertFalse(exlocal2 in mock.client_user_context_extensions) + self.assertFalse(exglobal2 in mock.client_user_context_extensions) + + mock.client_user_context_extensions = [] + client.interrupt_all() + self.assertFalse(exlocal in mock.client_user_context_extensions) + self.assertFalse(exglobal in mock.client_user_context_extensions) + self.assertFalse(exlocal2 in mock.client_user_context_extensions) + self.assertFalse(exglobal2 in mock.client_user_context_extensions) + + mock.client_user_context_extensions = [] + client.get_configs("foo", "bar") + self.assertFalse(exlocal in mock.client_user_context_extensions) + self.assertFalse(exglobal in mock.client_user_context_extensions) + self.assertFalse(exlocal2 in mock.client_user_context_extensions) + self.assertFalse(exglobal2 in mock.client_user_context_extensions) + finally: + client.close() + def test_interrupt_all(self): client = SparkConnectClient("sc://foo/;token=bar", use_reattachable_execute=False) mock = MockService(client._session_id) From f2f550dda131e314ffccf0ca994715e0f47f36d6 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 21 Nov 2025 18:29:11 +0900 Subject: [PATCH 183/400] [MINOR][PYTHON][TEST] test_install_spark` switch to Spark 3.5.7 ### What changes were proposed in this pull request? Similar to https://github.com/apache/spark/pull/51103 ### Why are the changes needed? Avoid downloading from Apache archive site, which is pretty slow and unstable. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GHA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53151 from pan3793/minor-pyspark-install-test. Authored-by: Cheng Pan Signed-off-by: Hyukjin Kwon (cherry picked from commit 00dda2c8d03c3112096ccfabbff7118d803b676e) Signed-off-by: Dongjoon Hyun --- python/pyspark/tests/test_install_spark.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/tests/test_install_spark.py b/python/pyspark/tests/test_install_spark.py index effbab6a90102..769876b46138b 100644 --- a/python/pyspark/tests/test_install_spark.py +++ b/python/pyspark/tests/test_install_spark.py @@ -32,7 +32,7 @@ class SparkInstallationTestCase(unittest.TestCase): def test_install_spark(self): # Test only one case. Testing this is expensive because it needs to download # the Spark distribution, ensure it is available at https://dlcdn.apache.org/spark/ - spark_version, hadoop_version, hive_version = checked_versions("3.5.6", "3", "2.3") + spark_version, hadoop_version, hive_version = checked_versions("3.5.7", "3", "2.3") with tempfile.TemporaryDirectory(prefix="test_install_spark") as tmp_dir: install_spark( From 8e1034a5bc2e7f68ba452583c877bfbcea41f04d Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 21 Nov 2025 08:24:01 -0800 Subject: [PATCH 184/400] [SPARK-54354][SQL] Fix Spark hanging when there's not enough JVM heap memory for broadcast hashed relation ### What changes were proposed in this pull request? A fix to let Spark throw OOM rather than hang when there's not enough JVM heap memory for broadcast hashed relation. The fix is done by passing the current JVM's heap size rather than `Long.MaxValue / 2` to create the temporary `UnifiedMemoryManager` for broadcasting. This is an optimal setting because if the size we passed is too large, i.e., the current `Long.MaxValue / 2`, it will cause hanging; if the size is smaller than the current JVM heap size, the OOM might be thrown too early even when there's room in memory for the newly created hashed relation. Before: ```scala new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, Long.MaxValue / 2, 1) ``` After: ```scala new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Runtime.getRuntime.maxMemory, Runtime.getRuntime.maxMemory / 2, 1) ``` ### Why are the changes needed? Report the error fast instead of hanging. ### Does this PR introduce _any_ user-facing change? In some scenarios where large unsafe hashed relations are allocated for broadcast hash join, user will see a meaningful OOM instead of hanging. Before (hangs): ``` 15:07:38.456 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (8589934592 bytes), try again. 15:07:38.501 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (8589934592 bytes), try again. 15:07:38.539 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (8589934592 bytes), try again. 15:07:38.580 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (8589934592 bytes), try again. 15:07:38.613 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (8589934592 bytes), try again. 15:07:38.647 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (8589934592 bytes), try again. ... ``` After (OOM): ``` An exception or error caused a run to abort: [UNABLE_TO_ACQUIRE_MEMORY] Unable to acquire 8589934592 bytes of memory, got 7194909081. SQLSTATE: 53200 org.apache.spark.memory.SparkOutOfMemoryError: [UNABLE_TO_ACQUIRE_MEMORY] Unable to acquire 8589934592 bytes of memory, got 7194909081. SQLSTATE: 53200 at org.apache.spark.errors.SparkCoreErrors$.outOfMemoryError(SparkCoreErrors.scala:456) at org.apache.spark.errors.SparkCoreErrors.outOfMemoryError(SparkCoreErrors.scala) at org.apache.spark.memory.MemoryConsumer.throwOom(MemoryConsumer.java:157) at org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:98) at org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:868) at org.apache.spark.unsafe.map.BytesToBytesMap.(BytesToBytesMap.java:202) at org.apache.spark.unsafe.map.BytesToBytesMap.(BytesToBytesMap.java:209) at org.apache.spark.sql.execution.joins.UnsafeHashedRelation$.apply(HashedRelation.scala:464) at org.apache.spark.sql.execution.joins.HashedRelationSuite.$anonfun$new$90(HashedRelationSuite.scala:760) ``` ### How was this patch tested? Using the following code to do a manual test since we don't want to add a test case that captures an OOM error: ```scala // The PR's practice to use `Runtime.getRuntime.maxMemory` as the maximum size. val umm = new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Runtime.getRuntime.maxMemory, Runtime.getRuntime.maxMemory / 2, 1) val mm = new TaskMemoryManager(umm, 0) val relations = mutable.ArrayBuffer[HashedRelation]() // We should finally see an OOM thrown since we are keeping allocating hashed relations. assertThrows[SparkOutOfMemoryError] { while (true) { // Allocates ~128 MiB each time. relations += UnsafeHashedRelation(Iterator.empty, Nil, 1 << 22, mm) } } // Releases the allocated memory. relations.foreach(_.close()) mm.cleanUpAllAllocatedMemory ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53065 from zhztheplayer/wip-54353-mm-hang. Authored-by: Hongze Zhang Signed-off-by: Dongjoon Hyun (cherry picked from commit ac69d935ddac432550167780ef0edb6dc055fd55) Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/joins/HashedRelation.scala | 12 ++++++------ .../sql/execution/joins/HashedRelationSuite.scala | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 85c1982905420..c67b55fd1d50c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -143,8 +143,8 @@ private[execution] object HashedRelation { new TaskMemoryManager( new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, + Runtime.getRuntime.maxMemory, + Runtime.getRuntime.maxMemory / 2, 1), 0) } @@ -401,8 +401,8 @@ private[joins] class UnsafeHashedRelation( val taskMemoryManager = new TaskMemoryManager( new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, + Runtime.getRuntime.maxMemory, + Runtime.getRuntime.maxMemory / 2, 1), 0) @@ -576,8 +576,8 @@ private[execution] final class LongToUnsafeRowMap( new TaskMemoryManager( new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, + Runtime.getRuntime.maxMemory, + Runtime.getRuntime.maxMemory / 2, 1), 0), 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index b88a76bbfb575..e46761f5cd048 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -42,8 +42,8 @@ import org.apache.spark.util.collection.CompactBuffer class HashedRelationSuite extends SharedSparkSession { val umm = new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue / 2, + Runtime.getRuntime.maxMemory, + Runtime.getRuntime.maxMemory / 2, 1) val mm = new TaskMemoryManager(umm, 0) From e48cb8cc69a9421bb8e7b438df31dd097c2d3529 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 21 Nov 2025 12:40:00 -0800 Subject: [PATCH 185/400] [SPARK-54448][SQL][DOCS] Fix docs and error message because CREATE TEMP VIEW does not support IF NOT EXISTS ### What changes were proposed in this pull request? Currently, Spark SQL supports `CREATE [ OR REPLACE ] [ [ GLOBAL ] TEMPORARY ] VIEW [ IF NOT EXISTS ] view_identifier ...` syntax, but the implementation forbids using `CREATE [ GLOBAL ] TEMPORARY VIEW` with `IF NOT EXISTS`, this is likely an incompleted feature. And this was also provided as an example in the docs, but actually does not work. https://spark.apache.org/docs/4.0.1/sql-ref-syntax-ddl-create-view.html image In addition, the error condition `TEMP_TABLE_OR_VIEW_ALREADY_EXISTS`'s message suggests "add the IF NOT EXISTS clause", which does not work either. ``` spark-sql (default)> CREATE GLOBAL TEMPORARY VIEW v AS SELECT 1; spark-sql (default)> CREATE GLOBAL TEMPORARY VIEW v AS SELECT 1; [TEMP_TABLE_OR_VIEW_ALREADY_EXISTS] Cannot create the temporary view `v` because it already exists. Choose a different name, drop or replace the existing view, or add the IF NOT EXISTS clause to tolerate pre-existing views. SQLSTATE: 42P07 ``` ### Why are the changes needed? Fix wrong docs and error condition message. ### Does this PR introduce _any_ user-facing change? It only updates the docs and error condition message, with no functionality changes. ### How was this patch tested? image ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53153 from pan3793/SPARK-54448. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 6c6dd0f5c7fe796f5e446bf9888f5908de4dabbe) Signed-off-by: Dongjoon Hyun --- common/utils/src/main/resources/error/error-conditions.json | 2 +- docs/sql-ref-syntax-ddl-create-view.md | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 555f310e512e3..b79e44a7984a0 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5919,7 +5919,7 @@ "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS" : { "message" : [ "Cannot create the temporary view because it already exists.", - "Choose a different name, drop or replace the existing view, or add the IF NOT EXISTS clause to tolerate pre-existing views." + "Choose a different name, drop or replace the existing view." ], "sqlState" : "42P07" }, diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index 21174f12300e3..2d832636b38fc 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -47,6 +47,7 @@ CREATE [ OR REPLACE ] [ [ GLOBAL ] TEMPORARY ] VIEW [ IF NOT EXISTS ] view_ident * **IF NOT EXISTS** Creates a view if it does not exist. + This clause is not supported for `TEMPORARY` views yet. * **view_identifier** @@ -86,8 +87,8 @@ CREATE OR REPLACE VIEW experienced_employee AS SELECT id, name FROM all_employee WHERE working_years > 5; --- Create a global temporary view `subscribed_movies` if it does not exist. -CREATE GLOBAL TEMPORARY VIEW IF NOT EXISTS subscribed_movies +-- Create a global temporary view `subscribed_movies`. +CREATE GLOBAL TEMPORARY VIEW subscribed_movies AS SELECT mo.member_id, mb.full_name, mo.movie_title FROM movies AS mo INNER JOIN members AS mb ON mo.member_id = mb.id; From 6d7b3907a9a4754630150a4dd533c8f35e9676b8 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Fri, 21 Nov 2025 12:41:56 -0800 Subject: [PATCH 186/400] [SPARK-54410][SQL] Fix read support for the variant logical type annotation ### What changes were proposed in this pull request? [This PR](https://github.com/apache/spark/pull/53005) introduced a fix where the Spark parquet writer would annotate variant columns with the parquet variant logical type. The PR had an ad-hoc fix on the reader side for validation. This PR formally allows Spark to read parquet files with the Variant logical type. The PR also introduces an unrelated fix in ParquetRowConverter to allow Spark to read variant columns regardless of which order the value and metadata fields are stored in. ### Why are the changes needed? The variant logical type annotation has formally been adopted as part of the parquet spec in is part of the parquet-java 1.16.0 library. Therefore, Spark should be able to read files containing data annotated as such. ### Does this PR introduce _any_ user-facing change? Yes, it allows users to read parquet files with the variant logical type annotation. ### How was this patch tested? Existing test from [this PR](https://github.com/apache/spark/pull/53005) where we wrote data of the variant logical type and tested read using an ad-hoc solution. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53120 from harshmotw-db/harshmotw-db/variant_annotation_write. Authored-by: Harsh Motwani Signed-off-by: Dongjoon Hyun (cherry picked from commit da7389bb81ab9756c70a5aa168c5128632c3b31e) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/internal/SQLConf.scala | 13 +- .../parquet/ParquetRowConverter.scala | 36 ++-- .../parquet/ParquetSchemaConverter.scala | 38 +++- .../parquet/SparkShreddingUtils.scala | 4 +- .../ParquetVariantShreddingSuite.scala | 175 ++++++++++++------ 5 files changed, 182 insertions(+), 84 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 940460620da09..5ae5fa33a721d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1600,6 +1600,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PARQUET_IGNORE_VARIANT_ANNOTATION = + buildConf("spark.sql.parquet.ignoreVariantAnnotation") + .internal() + .doc("When true, ignore the variant logical type annotation and treat the Parquet " + + "column in the same way as the underlying struct type") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + val PARQUET_FIELD_ID_READ_ENABLED = buildConf("spark.sql.parquet.fieldId.read.enabled") .doc("Field ID is a native field of the Parquet schema spec. When enabled, Parquet readers " + @@ -5508,7 +5517,7 @@ object SQLConf { "When false, it only reads unshredded variant.") .version("4.0.0") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val PUSH_VARIANT_INTO_SCAN = buildConf("spark.sql.variant.pushVariantIntoScan") @@ -7685,6 +7694,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def parquetAnnotateVariantLogicalType: Boolean = getConf(PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE) + def parquetIgnoreVariantAnnotation: Boolean = getConf(SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION) + def ignoreMissingParquetFieldId: Boolean = getConf(SQLConf.IGNORE_MISSING_PARQUET_FIELD_ID) def legacyParquetNanosAsLong: Boolean = getConf(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index d708a19dd1ac7..271a1485dfd34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -876,7 +876,11 @@ private[parquet] class ParquetRowConverter( } } - /** Parquet converter for unshredded Variant */ + /** + * Parquet converter for unshredded Variant. We use this converter when the + * `spark.sql.variant.allowReadingShredded` config is set to false. This option just exists to + * fall back to legacy logic which will eventually be removed. + */ private final class ParquetUnshreddedVariantConverter( parquetType: GroupType, updater: ParentContainerUpdater) @@ -890,29 +894,27 @@ private[parquet] class ParquetRowConverter( // We may allow more than two children in the future, so consider this unsupported. throw QueryCompilationErrors.invalidVariantWrongNumFieldsError() } - val valueAndMetadata = Seq("value", "metadata").map { colName => + val Seq(value, metadata) = Seq("value", "metadata").map { colName => val idx = (0 until parquetType.getFieldCount()) - .find(parquetType.getFieldName(_) == colName) - if (idx.isEmpty) { - throw QueryCompilationErrors.invalidVariantMissingFieldError(colName) - } - val child = parquetType.getType(idx.get) + .find(parquetType.getFieldName(_) == colName) + .getOrElse(throw QueryCompilationErrors.invalidVariantMissingFieldError(colName)) + val child = parquetType.getType(idx) if (!child.isPrimitive || child.getRepetition != Type.Repetition.REQUIRED || - child.asPrimitiveType().getPrimitiveTypeName != BINARY) { + child.asPrimitiveType().getPrimitiveTypeName != BINARY) { throw QueryCompilationErrors.invalidVariantNullableOrNotBinaryFieldError(colName) } - child + idx } - Array( - // Converter for value - newConverter(valueAndMetadata(0), BinaryType, new ParentContainerUpdater { + val result = new Array[Converter with HasParentContainerUpdater](2) + result(value) = + newConverter(parquetType.getType(value), BinaryType, new ParentContainerUpdater { override def set(value: Any): Unit = currentValue = value - }), - - // Converter for metadata - newConverter(valueAndMetadata(1), BinaryType, new ParentContainerUpdater { + }) + result(metadata) = + newConverter(parquetType.getType(metadata), BinaryType, new ParentContainerUpdater { override def set(value: Any): Unit = currentMetadata = value - })) + }) + result } override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index d7110c7369993..9e6f4447ca792 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -58,7 +58,9 @@ class ParquetToSparkSchemaConverter( caseSensitive: Boolean = SQLConf.CASE_SENSITIVE.defaultValue.get, inferTimestampNTZ: Boolean = SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.defaultValue.get, nanosAsLong: Boolean = SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValue.get, - useFieldId: Boolean = SQLConf.PARQUET_FIELD_ID_READ_ENABLED.defaultValue.get) { + useFieldId: Boolean = SQLConf.PARQUET_FIELD_ID_READ_ENABLED.defaultValue.get, + val ignoreVariantAnnotation: Boolean = + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.defaultValue.get) { def this(conf: SQLConf) = this( assumeBinaryIsString = conf.isParquetBinaryAsString, @@ -66,7 +68,8 @@ class ParquetToSparkSchemaConverter( caseSensitive = conf.caseSensitiveAnalysis, inferTimestampNTZ = conf.parquetInferTimestampNTZEnabled, nanosAsLong = conf.legacyParquetNanosAsLong, - useFieldId = conf.parquetFieldIdReadEnabled) + useFieldId = conf.parquetFieldIdReadEnabled, + ignoreVariantAnnotation = conf.parquetIgnoreVariantAnnotation) def this(conf: Configuration) = this( assumeBinaryIsString = conf.get(SQLConf.PARQUET_BINARY_AS_STRING.key).toBoolean, @@ -75,7 +78,9 @@ class ParquetToSparkSchemaConverter( inferTimestampNTZ = conf.get(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key).toBoolean, nanosAsLong = conf.get(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key).toBoolean, useFieldId = conf.getBoolean(SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key, - SQLConf.PARQUET_FIELD_ID_READ_ENABLED.defaultValue.get)) + SQLConf.PARQUET_FIELD_ID_READ_ENABLED.defaultValue.get), + ignoreVariantAnnotation = conf.getBoolean(SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key, + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.defaultValue.get)) /** * Converts Parquet [[MessageType]] `parquetSchema` to a Spark SQL [[StructType]]. @@ -202,15 +207,17 @@ class ParquetToSparkSchemaConverter( case primitiveColumn: PrimitiveColumnIO => convertPrimitiveField(primitiveColumn, targetType) case groupColumn: GroupColumnIO if targetType.contains(VariantType) => if (SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED)) { - val col = convertGroupField(groupColumn) + // We need the underlying file type regardless of the config. + val col = convertGroupField(groupColumn, ignoreVariantAnnotation = true) col.copy(sparkType = VariantType, variantFileType = Some(col)) } else { convertVariantField(groupColumn) } case groupColumn: GroupColumnIO if targetType.exists(VariantMetadata.isVariantStruct) => - val col = convertGroupField(groupColumn) + val col = convertGroupField(groupColumn, ignoreVariantAnnotation = true) col.copy(sparkType = targetType.get, variantFileType = Some(col)) - case groupColumn: GroupColumnIO => convertGroupField(groupColumn, targetType) + case groupColumn: GroupColumnIO => + convertGroupField(groupColumn, ignoreVariantAnnotation, targetType) } } @@ -349,6 +356,7 @@ class ParquetToSparkSchemaConverter( private def convertGroupField( groupColumn: GroupColumnIO, + ignoreVariantAnnotation: Boolean, sparkReadType: Option[DataType] = None): ParquetColumn = { val field = groupColumn.getType.asGroupType() @@ -373,9 +381,21 @@ class ParquetToSparkSchemaConverter( Option(field.getLogicalTypeAnnotation).fold( convertInternal(groupColumn, sparkReadType.map(_.asInstanceOf[StructType]))) { - // Temporary workaround to read Shredded variant data - case v: VariantLogicalTypeAnnotation if v.getSpecVersion == 1 && sparkReadType.isEmpty => - convertInternal(groupColumn, None) + case v: VariantLogicalTypeAnnotation if v.getSpecVersion == 1 => + if (ignoreVariantAnnotation) { + convertInternal(groupColumn) + } else { + ParquetSchemaConverter.checkConversionRequirement( + sparkReadType.forall(_.isInstanceOf[VariantType]), + s"Invalid Spark read type: expected $field to be variant type but found " + + s"${if (sparkReadType.isEmpty) { "None" } else {sparkReadType.get.sql} }") + if (SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED)) { + val col = convertInternal(groupColumn) + col.copy(sparkType = VariantType, variantFileType = Some(col)) + } else { + convertVariantField(groupColumn) + } + } // A Parquet list is represented as a 3-level structure: // diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index 1132f074f29d1..ca2defffba913 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -646,7 +646,9 @@ case object SparkShreddingUtils { def parquetTypeToSparkType(parquetType: ParquetType): DataType = { val messageType = ParquetTypes.buildMessage().addField(parquetType).named("foo") val column = new ColumnIOFactory().getColumnIO(messageType) - new ParquetToSparkSchemaConverter().convertField(column.getChild(0)).sparkType + // We need the underlying file type regardless of the ignoreVariantAnnotation config. + val converter = new ParquetToSparkSchemaConverter(ignoreVariantAnnotation = true) + converter.convertField(column.getChild(0)).sparkType } class SparkShreddedResult(schema: VariantSchema) extends VariantShreddingWriter.ShreddedResult { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala index a9ec5e161f348..77140c1a91ee0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala @@ -28,7 +28,8 @@ import org.apache.parquet.hadoop.util.HadoopInputFile import org.apache.parquet.schema.{LogicalTypeAnnotation, PrimitiveType, Type} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType import org.apache.spark.sql.test.SharedSparkSession @@ -160,64 +161,126 @@ class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with Share Seq(false, true).foreach { annotateVariantLogicalType => Seq(false, true).foreach { shredVariant => Seq(false, true).foreach { allowReadingShredded => - withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> shredVariant.toString, - SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key -> shredVariant.toString, - SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> - (allowReadingShredded || shredVariant).toString, - SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key -> - annotateVariantLogicalType.toString) { - def validateAnnotation(g: Type): Unit = { - if (annotateVariantLogicalType) { - assert(g.getLogicalTypeAnnotation == LogicalTypeAnnotation.variantType(1)) - } else { - assert(g.getLogicalTypeAnnotation == null) + Seq(false, true).foreach { ignoreVariantAnnotation => + withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> shredVariant.toString, + SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key -> shredVariant.toString, + SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> + (allowReadingShredded || shredVariant).toString, + SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key -> + annotateVariantLogicalType.toString, + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> ignoreVariantAnnotation.toString) { + def validateAnnotation(g: Type): Unit = { + if (annotateVariantLogicalType) { + assert(g.getLogicalTypeAnnotation == LogicalTypeAnnotation.variantType(1)) + } else { + assert(g.getLogicalTypeAnnotation == null) + } + } + withTempDir { dir => + // write parquet file + val df = spark.sql( + """ + | select + | id * 2 i, + | to_variant_object(named_struct('id', id)) v, + | named_struct('i', (id * 2)::string, + | 'nv', to_variant_object(named_struct('id', 30 + id))) ns, + | array(to_variant_object(named_struct('id', 10 + id))) av, + | map('v2', to_variant_object(named_struct('id', 20 + id))) mv + | from range(0,3,1,1)""".stripMargin) + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + val file = dir.listFiles().find(_.getName.endsWith(".parquet")).get + val parquetFilePath = file.getAbsolutePath + val inputFile = HadoopInputFile.fromPath(new Path(parquetFilePath), + new Configuration()) + val reader = ParquetFileReader.open(inputFile) + val footer = reader.getFooter + val schema = footer.getFileMetaData.getSchema + val vGroup = schema.getType(schema.getFieldIndex("v")) + validateAnnotation(vGroup) + assert(vGroup.asGroupType().getFields.asScala.toSeq + .exists(_.getName == "typed_value") == shredVariant) + val nsGroup = schema.getType(schema.getFieldIndex("ns")).asGroupType() + val nvGroup = nsGroup.getType(nsGroup.getFieldIndex("nv")) + validateAnnotation(nvGroup) + val avGroup = schema.getType(schema.getFieldIndex("av")).asGroupType() + val avList = avGroup.getType(avGroup.getFieldIndex("list")).asGroupType() + val avElement = avList.getType(avList.getFieldIndex("element")) + validateAnnotation(avElement) + val mvGroup = schema.getType(schema.getFieldIndex("mv")).asGroupType() + val mvList = mvGroup.getType(mvGroup.getFieldIndex("key_value")).asGroupType() + val mvValue = mvList.getType(mvList.getFieldIndex("value")) + validateAnnotation(mvValue) + // verify result + val result = spark.read.format("parquet") + .schema("v variant, ns struct, av array, " + + "mv map") + .load(dir.getAbsolutePath) + .selectExpr("v:id::int i1", "ns.nv:id::int i2", "av[0]:id::int i3", + "mv['v2']:id::int i4") + checkAnswer(result, Array(Row(0, 30, 10, 20), Row(1, 31, 11, 21), + Row(2, 32, 12, 22))) + reader.close() } } - withTempDir { dir => - // write parquet file - val df = spark.sql( - """ - | select - | id * 2 i, - | to_variant_object(named_struct('id', id)) v, - | named_struct('i', (id * 2)::string, - | 'nv', to_variant_object(named_struct('id', 30 + id))) ns, - | array(to_variant_object(named_struct('id', 10 + id))) av, - | map('v2', to_variant_object(named_struct('id', 20 + id))) mv - | from range(0,3,1,1)""".stripMargin) - df.write.mode("overwrite").parquet(dir.getAbsolutePath) - val file = dir.listFiles().find(_.getName.endsWith(".parquet")).get - val parquetFilePath = file.getAbsolutePath - val inputFile = HadoopInputFile.fromPath(new Path(parquetFilePath), - new Configuration()) - val reader = ParquetFileReader.open(inputFile) - val footer = reader.getFooter - val schema = footer.getFileMetaData.getSchema - val vGroup = schema.getType(schema.getFieldIndex("v")) - validateAnnotation(vGroup) - assert(vGroup.asGroupType().getFields.asScala.toSeq - .exists(_.getName == "typed_value") == shredVariant) - val nsGroup = schema.getType(schema.getFieldIndex("ns")).asGroupType() - val nvGroup = nsGroup.getType(nsGroup.getFieldIndex("nv")) - validateAnnotation(nvGroup) - val avGroup = schema.getType(schema.getFieldIndex("av")).asGroupType() - val avList = avGroup.getType(avGroup.getFieldIndex("list")).asGroupType() - val avElement = avList.getType(avList.getFieldIndex("element")) - validateAnnotation(avElement) - val mvGroup = schema.getType(schema.getFieldIndex("mv")).asGroupType() - val mvList = mvGroup.getType(mvGroup.getFieldIndex("key_value")).asGroupType() - val mvValue = mvList.getType(mvList.getFieldIndex("value")) - validateAnnotation(mvValue) - // verify result - val result = spark.read.format("parquet") - .schema("v variant, ns struct, av array, " + - "mv map") - .load(dir.getAbsolutePath) - .selectExpr("v:id::int i1", "ns.nv:id::int i2", "av[0]:id::int i3", - "mv['v2']:id::int i4") - checkAnswer(result, Array(Row(0, 30, 10, 20), Row(1, 31, 11, 21), Row(2, 32, 12, 22))) - reader.close() + } + } + } + } + } + + test("variant logical type annotation - ignore variant annotation") { + Seq(true, false).foreach { ignoreVariantAnnotation => + withSQLConf(SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key -> "true", + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> ignoreVariantAnnotation.toString + ) { + withTempDir { dir => + // write parquet file + val df = spark.sql( + """ + | select + | id * 2 i, + | 1::variant v, + | named_struct('i', (id * 2)::string, 'nv', 1::variant) ns, + | array(1::variant) av, + | map('v2', 1::variant) mv + | from range(0,1,1,1)""".stripMargin) + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + // verify result + val normal_result = spark.read.format("parquet") + .schema("v variant, ns struct, av array, " + + "mv map") + .load(dir.getAbsolutePath) + .selectExpr("v::int i1", "ns.nv::int i2", "av[0]::int i3", + "mv['v2']::int i4") + checkAnswer(normal_result, Array(Row(1, 1, 1, 1))) + val struct_result = spark.read.format("parquet") + .schema("v struct, " + + "ns struct>, " + + "av array>, " + + "mv map>") + .load(dir.getAbsolutePath) + .selectExpr("v", "ns.nv", "av[0]", "mv['v2']") + if (ignoreVariantAnnotation) { + checkAnswer( + struct_result, + Seq(Row( + Row(Array[Byte](12, 1), Array[Byte](1, 0, 0)), + Row(Array[Byte](12, 1), Array[Byte](1, 0, 0)), + Row(Array[Byte](12, 1), Array[Byte](1, 0, 0)), + Row(Array[Byte](12, 1), Array[Byte](1, 0, 0)) + )) + ) + } else { + val exception = intercept[SparkException]{ + struct_result.collect() } + checkError( + exception = exception.getCause.asInstanceOf[AnalysisException], + condition = "_LEGACY_ERROR_TEMP_3071", + parameters = Map("msg" -> "Invalid Spark read type[\\s\\S]*"), + matchPVals = true + ) } } } From 22816f2a152d6acdadad793633b8fc17b1d7a693 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Fri, 21 Nov 2025 13:17:24 -0800 Subject: [PATCH 187/400] [SPARK-54087][CORE] Spark Executor launch task failed should return task killed message MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? image When not enable speculation ,executor launch task failed caused by OOM, image DAGScheduler won't know task failed and won't scheduler again, causing application stuck image-2025-06-23-12-13-10-770 ### Why are the changes needed? Some case will causing application stuck missing SLA ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT + MT Test code image ``` import org.apache.spark.sql.functions.udf import java.util.concurrent.TimeUnit// 定义一个会sleep的UDF val sleepUdf = udf((value: String) => { TimeUnit.SECONDS.sleep(10) value }) val df = spark.range(0, 2500).repartition(25).withColumn("value", (col("id") % 10).cast("string")).withColumn("value", sleepUdf(col("value")))df.createOrReplaceTempView("test_table") val result = spark.sql("SELECT value, COUNT(*) as cnt FROM test_table GROUP BY value ORDER BY cnt DESC") result.show() ``` image-2025-07-23-14-44-53-911 image-2025-07-23-14-45-12-522 task failed and re-scheduled ### Was this patch authored or co-authored using generative AI tooling? No Closes #52792 from AngersZhuuuu/SPARK-54087. Authored-by: Angerszhuuuu Signed-off-by: Dongjoon Hyun (cherry picked from commit 7c279e54257d56d01dffa5b98a7a9945bb371fd3) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/executor/Executor.scala | 23 ++++++++- .../apache/spark/executor/ExecutorSuite.scala | 50 ++++++++++++++++++- 2 files changed, 71 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index eb45e810d4255..fc22107e008be 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -381,7 +381,28 @@ private[spark] class Executor( tr.kill(killMark._1, killMark._2) killMarks.remove(taskId) } - threadPool.execute(tr) + try { + threadPool.execute(tr) + } catch { + case t: Throwable => + try { + logError(log"Executor launch task ${MDC(TASK_NAME, taskDescription.name)} failed," + + log" reason: ${MDC(REASON, t.getMessage)}") + context.statusUpdate( + taskDescription.taskId, + TaskState.FAILED, + env.closureSerializer.newInstance().serialize(new ExceptionFailure(t, Seq.empty))) + } catch { + case oom: OutOfMemoryError => + logError(log"Executor update launching task ${MDC(TASK_NAME, taskDescription.name)} " + + log"failed status failed, reason: ${MDC(REASON, oom.getMessage)}") + System.exit(SparkExitCode.OOM) + case t: Throwable => + logError(log"Executor update launching task ${MDC(TASK_NAME, taskDescription.name)} " + + log"failed status failed, reason: ${MDC(REASON, t.getMessage)}") + System.exit(-1) + } + } if (decommissioned) { log.error(s"Launching a task while in decommissioned state.") } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 6f525cf8b898a..dd9884bffb285 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -22,7 +22,7 @@ import java.lang.Thread.UncaughtExceptionHandler import java.net.URL import java.nio.ByteBuffer import java.util.{HashMap, Properties} -import java.util.concurrent.{CountDownLatch, TimeUnit} +import java.util.concurrent.{CountDownLatch, ThreadPoolExecutor, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean import scala.collection.immutable @@ -603,6 +603,54 @@ class ExecutorSuite extends SparkFunSuite } } + test("SPARK-54087: launchTask should return task killed message when threadPool.execute fails") { + val conf = new SparkConf + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + val serializedTask = serializer.newInstance().serialize(new FakeTask(0, 0)) + val taskDescription = createFakeTaskDescription(serializedTask) + + val mockExecutorBackend = mock[ExecutorBackend] + val statusCaptor = ArgumentCaptor.forClass(classOf[ByteBuffer]) + + withExecutor("id", "localhost", env) { executor => + // Use reflection to replace threadPool with a mock that throws an exception + val executorClass = classOf[Executor] + val threadPoolField = executorClass.getDeclaredField("threadPool") + threadPoolField.setAccessible(true) + val originalThreadPool = threadPoolField.get(executor).asInstanceOf[ThreadPoolExecutor] + + // Create a mock ThreadPoolExecutor that throws an exception when execute is called + val mockThreadPool = mock[ThreadPoolExecutor] + val testException = new OutOfMemoryError("unable to create new native thread") + when(mockThreadPool.execute(any[Runnable])).thenThrow(testException) + threadPoolField.set(executor, mockThreadPool) + + try { + // Launch the task - this should catch the exception and send statusUpdate + executor.launchTask(mockExecutorBackend, taskDescription) + + // Verify that statusUpdate was called with FAILED state + verify(mockExecutorBackend).statusUpdate( + meq(taskDescription.taskId), + meq(TaskState.FAILED), + statusCaptor.capture() + ) + + // Verify that the exception was correctly serialized + val failureData = statusCaptor.getValue + val failReason = serializer.newInstance() + .deserialize[ExceptionFailure](failureData) + assert(failReason.exception.isDefined) + assert(failReason.exception.get.isInstanceOf[OutOfMemoryError]) + assert(failReason.exception.get.getMessage === "unable to create new native thread") + } finally { + // Restore the original threadPool + threadPoolField.set(executor, originalThreadPool) + } + } + } + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { val mockEnv = mock[SparkEnv] val mockRpcEnv = mock[RpcEnv] From 0669bc8905df69156b99dddc65d5f4469775423e Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Fri, 21 Nov 2025 13:47:53 -0800 Subject: [PATCH 188/400] [SPARK-53797][SQL] Fix `FileStreamSource.takeFilesUntilMax` to use `zipWithIndex` to avoid `indices` usage ### What changes were proposed in this pull request? Fixes a performance issue with the new `maxBytesPerTrigger` option for file stream sources introduced in https://github.com/apache/spark/pull/44636. This changes the iteration of files for calculating offsets when `maxBytesPerTrigger` is used with file stream sources from list indexing to iteration. ### Why are the changes needed? We tried out this new option and found streams reading for tables with a lot of files (in the millions) were spending hours trying to construct batches. After looking at the thread dump for these, I could see a Scala immutable.List was what the `files` object was stored as, which is a linked list under the hood with `O(n)` lookup time by index, making the `takesFilesUntilMax` method a `O(n^2)` operation. Instead the list should simply be iterated over which makes it a simple `O(n)` operation overall. ### Does this PR introduce _any_ user-facing change? No, just performance. ### How was this patch tested? No new tests, purely a performance improvement. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52515 from Kimahriman/max-bytes-per-trigger-iter. Authored-by: Adam Binford Signed-off-by: Dongjoon Hyun (cherry picked from commit a870ea459712ad5e8d887e6ca8b7bb16b79d0130) Signed-off-by: Dongjoon Hyun --- .../sql/execution/streaming/runtime/FileStreamSource.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala index d5503f1c247da..9847bd9d76448 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala @@ -149,8 +149,7 @@ class FileStreamSource( var rSize = BigInt(0) val lFiles = ArrayBuffer[NewFileEntry]() val rFiles = ArrayBuffer[NewFileEntry]() - for (i <- files.indices) { - val file = files(i) + files.zipWithIndex.foreach { case (file, i) => val newSize = lSize + file.size if (i == 0 || rFiles.isEmpty && newSize <= Long.MaxValue && newSize <= maxSize) { lSize += file.size From e32d0e3bdd2b1eb99017dadabbcf73ec7cbdbb51 Mon Sep 17 00:00:00 2001 From: Richard Chen Date: Fri, 21 Nov 2025 13:54:02 -0800 Subject: [PATCH 189/400] [SPARK-54427][SQL] Allow ColumnarRow to call `copy` with variant types ### What changes were proposed in this pull request? add a `Variant` matching `if` statement when trying to copy columnar row. ### Why are the changes needed? currently, trying to copy a columnar row with a variant type will crash the query. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? added UT ### Was this patch authored or co-authored using generative AI tooling? no Closes #53137 from richardc-db/fix_columnar_row_variant_copy. Authored-by: Richard Chen Signed-off-by: Dongjoon Hyun (cherry picked from commit 0d99d6d852cd859aaac8cbc0211093ccc9f98287) Signed-off-by: Dongjoon Hyun --- .../spark/sql/vectorized/ColumnarRow.java | 2 + .../vectorized/ColumnVectorSuite.scala | 39 ++++++++++++++++++- 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index d9e65afe1cb00..656c5f8a8f30e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java @@ -91,6 +91,8 @@ public InternalRow copy() { row.update(i, getArray(i).copy()); } else if (pdt instanceof PhysicalMapType) { row.update(i, getMap(i).copy()); + } else if (pdt instanceof PhysicalVariantType) { + row.update(i, getVariant(i)); } else { throw new RuntimeException("Not implemented. " + dt); } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 966e892dc4043..3be6c14eb4d3c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.execution.columnar.compression.ColumnBuilderHelper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarArray -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import org.apache.spark.util.ArrayImplicits._ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { @@ -166,6 +166,43 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { } } + testVectors("variant", 3, new StructType().add("v", VariantType)) { structVector => + val variantCol = structVector.getChild(0) + val valueChild = variantCol.getChild(0) + val metadataChild = variantCol.getChild(1) + + variantCol.putNotNull(0) + valueChild.appendByteArray(Array[Byte](1, 2, 3), 0, 3) + metadataChild.appendByteArray(Array[Byte](10, 11), 0, 2) + + variantCol.putNotNull(1) + valueChild.appendByteArray(Array[Byte](4, 5), 0, 2) + metadataChild.appendByteArray(Array[Byte](12, 13, 14), 0, 3) + + variantCol.putNull(2) + valueChild.appendNull() + metadataChild.appendNull() + + (0 until 3).foreach { i => + val row = structVector.getStruct(i) + val rowCopy = row.copy() + + if (i < 2) { + assert(!row.isNullAt(0)) + assert(!rowCopy.isNullAt(0)) + + val originalVariant = row.get(0, VariantType).asInstanceOf[VariantVal] + val copiedVariant = rowCopy.get(0, VariantType).asInstanceOf[VariantVal] + + assert(java.util.Arrays.equals(originalVariant.getValue, copiedVariant.getValue)) + assert(java.util.Arrays.equals(originalVariant.getMetadata, copiedVariant.getMetadata)) + } else { + assert(row.isNullAt(0)) + assert(rowCopy.isNullAt(0)) + } + } + } + testVectors("float", 10, FloatType) { testVector => (0 until 10).foreach { i => testVector.appendFloat(i.toFloat) From 23140962c218fd24d5a17c1b5ffec541095449e2 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Fri, 21 Nov 2025 14:30:48 -0800 Subject: [PATCH 190/400] [SPARK-54439][SQL] KeyGroupedPartitioning and join key size mismatch ### What changes were proposed in this pull request? Fix `KeyGroupedShuffleSpec.createPartitioning()` as clustering required at the other side of the join might contain more clustering expressions than the number of expressions in the shuffle spec's `KeyGroupedPartitioning`, so simply zipping them is not correct. ### Why are the changes needed? Fix a correctness issue due to wrong partitioning on the shuffle side. ### Does this PR introduce _any_ user-facing change? Yes, it fixes the query. ### How was this patch tested? Added new UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53142 from peter-toth/SPARK-54439-keygroupedpartitioning-and-join-key-size-mismatch. Authored-by: Peter Toth Signed-off-by: Dongjoon Hyun (cherry picked from commit 05602d57d9779855991c1447568c259aecdb546e) Signed-off-by: Dongjoon Hyun --- .../plans/physical/partitioning.scala | 11 ++-- .../KeyGroupedPartitioningSuite.scala | 52 +++++++++++++++++++ 2 files changed, 59 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 1cbb49c7a1f73..101d13c6b580c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -943,10 +943,13 @@ case class KeyGroupedShuffleSpec( } override def createPartitioning(clustering: Seq[Expression]): Partitioning = { - val newExpressions: Seq[Expression] = clustering.zip(partitioning.expressions).map { - case (c, e: TransformExpression) => TransformExpression( - e.function, Seq(c), e.numBucketsOpt) - case (c, _) => c + assert(clustering.size == distribution.clustering.size, + "Required distributions of join legs should be the same size.") + + val newExpressions = partitioning.expressions.zip(keyPositions).map { + case (te: TransformExpression, positionSet) => + te.copy(children = te.children.map(_ => clustering(positionSet.head))) + case (_, positionSet) => clustering(positionSet.head) } KeyGroupedPartitioning(newExpressions, partitioning.numPartitions, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index aba866e96b5c9..7c830bf6c6e18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -2771,4 +2771,56 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { } } } + + test("SPARK-54439: KeyGroupedPartitioning and join key size mismatch") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true") { + // `time` and `item_id` in the required `ClusteredDistribution` for `purchases`, but `item` is + // storage partitioned only by `id` + val df = createJoinTestDF(Seq("arrive_time" -> "time", "id" -> "item_id")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.size == 1, "only shuffle one side not report partitioning") + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0))) + } + } + + test("SPARK-54439: KeyGroupedPartitioning with transform and join key size mismatch") { + // Do not use `bucket()` in "one side partition" tests as its implementation in + // `InMemoryBaseTable` conflicts with `BucketFunction` + val items_partitions = Array(years("arrive_time")) + createTable(items, itemsColumns, items_partitions) + + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'bb', 10.0, cast('2021-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2021-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2021-02-01' as timestamp))") + + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true") { + // `item_id` and `time` in the required `ClusteredDistribution` for `purchases`, but `item` is + // storage partitioned only by `year(arrive_time)` + val df = createJoinTestDF(Seq("id" -> "item_id", "arrive_time" -> "time")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.size == 1, "only shuffle one side not report partitioning") + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0))) + } + } } From 296e6820eddcf2adc42a3ca7aa8ebcf387260f08 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 21 Nov 2025 14:55:16 -0800 Subject: [PATCH 191/400] [SPARK-54153][PYTHON][TESTS][FOLLOWUP] Skip `test_perf_profiler_data_source` if `pyarrow` is absent ### What changes were proposed in this pull request? This PR aims to skip `test_perf_profiler_data_source` if `pyarrow` is absent. ### Why are the changes needed? To recover the failed `PyPy` CIs. - https://github.com/apache/spark/actions/workflows/build_python_pypy3.10.yml - https://github.com/apache/spark/actions/runs/19574648782 - https://github.com/apache/spark/actions/runs/19574648782/job/56056836234 ``` ====================================================================== ERROR: test_perf_profiler_data_source (pyspark.sql.tests.test_udf_profiler.UDFProfiler2Tests) ---------------------------------------------------------------------- Traceback (most recent call last): File "/__w/spark/spark/python/pyspark/sql/tests/test_udf_profiler.py", line 609, in test_perf_profiler_data_source self.spark.read.format("TestDataSource").load().collect() File "/__w/spark/spark/python/pyspark/sql/classic/dataframe.py", line 469, in collect sock_info = self._jdf.collectToPython() File "/__w/spark/spark/python/lib/py4j-0.10.9.9-src.zip/py4j/java_gateway.py", line 1362, in __call__ return_value = get_return_value( File "/__w/spark/spark/python/pyspark/errors/exceptions/captured.py", line 263, in deco return f(*a, **kw) File "/__w/spark/spark/python/lib/py4j-0.10.9.9-src.zip/py4j/protocol.py", line 327, in get_return_value raise Py4JJavaError( py4j.protocol.Py4JJavaError: An error occurred while calling o235.collectToPython. : org.apache.spark.SparkException: Error from python worker: Traceback (most recent call last): File "/usr/local/pypy/pypy3.10/lib/pypy3.10/runpy.py", line 199, in _run_module_as_main return _run_code(code, main_globals, None, File "/usr/local/pypy/pypy3.10/lib/pypy3.10/runpy.py", line 86, in _run_code exec(code, run_globals) File "/__w/spark/spark/python/lib/pyspark.zip/pyspark/daemon.py", line 37, in File "/usr/local/pypy/pypy3.10/lib/pypy3.10/importlib/__init__.py", line 126, in import_module return _bootstrap._gcd_import(name[level:], package, level) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "", line 1050, in _gcd_import File "", line 1027, in _find_and_load File "", line 1006, in _find_and_load_unlocked File "", line 688, in _load_unlocked File "/frozen importlib._bootstrap_external", line 897, in exec_module File "", line 241, in _call_with_frames_removed File "/__w/spark/spark/python/lib/pyspark.zip/pyspark/sql/worker/plan_data_source_read.py", line 21, in import pyarrow as pa ModuleNotFoundError: No module named 'pyarrow' ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53162 from dongjoon-hyun/SPARK-54153. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 9b0b1ce2d628f18c5dbe85c0de9884960d50f71b) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_udf_profiler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py index 37f4a70fabd27..e6a7bf40b9454 100644 --- a/python/pyspark/sql/tests/test_udf_profiler.py +++ b/python/pyspark/sql/tests/test_udf_profiler.py @@ -585,6 +585,7 @@ def summarize(left, right): for id in self.profile_results: self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2) + @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) def test_perf_profiler_data_source(self): class TestDataSourceReader(DataSourceReader): def __init__(self, schema): From eb0bb7474bda95d187cc86043a9434bdae46193e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 21 Nov 2025 17:42:26 -0800 Subject: [PATCH 192/400] [SPARK-54455][CORE][TESTS] Disable a flaky `unmanaged memory tracking with off-heap memory enabled` test in MacOS CI ### What changes were proposed in this pull request? This PR aims to disable a flaky `unmanaged memory tracking with off-heap memory enabled` test in MacOS CI ### Why are the changes needed? This test suite passed locally, but has been flaky in MacOS GitHub Action environment. - https://github.com/apache/spark/actions/runs/19442963476/job/55985972662 ``` - unmanaged memory tracking with off-heap memory enabled *** FAILED *** 1400 was not less than or equal to 1200 Off-heap memory should be reduced by unmanaged usage (UnifiedMemoryManagerSuite.scala:592) ``` - https://github.com/apache/spark/actions/runs/19582140886/job/56082446144 ``` - unmanaged memory tracking with off-heap memory enabled *** FAILED *** mm.acquireStorageMemory(UnifiedMemoryManagerSuite.this.dummyBlock, 1100L, OFF_HEAP) was true (UnifiedMemoryManagerSuite.scala:600) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53165 from dongjoon-hyun/SPARK-54455. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 371b00ad422504b3462779e4867d45e660e3075e) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/memory/UnifiedMemoryManagerSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 9c74f2fdd459b..9f0e622b1d515 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.storage.TestBlockId import org.apache.spark.storage.memory.MemoryStore +import org.apache.spark.util.Utils class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTester { private val dummyBlock = TestBlockId("--") @@ -554,6 +555,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes } test("unmanaged memory tracking with off-heap memory enabled") { + assume(!Utils.isMacOnAppleSilicon) val maxOnHeapMemory = 1000L val maxOffHeapMemory = 1500L val taskAttemptId = 0L From 70b07eca33ac08f9f4a674f6c45190d9e1295ded Mon Sep 17 00:00:00 2001 From: antban Date: Tue, 18 Nov 2025 13:52:13 -0800 Subject: [PATCH 193/400] [SPARK-54349][PYTHON] Refactor code a bit to simplify faulthandler integration extension ### What changes were proposed in this pull request? There are many places where pyspark is trying to integrate with faulthandler and use the same functionality to dump stack traces/record thread dumps. In order to reduce the complexity of the integration and ease the extension of integration it makes sense to technically refactor the code to use the same code in all the places. ### Why are the changes needed? Improves developer experience. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By existing unit tests ### Was this patch authored or co-authored using generative AI tooling? Closes #53016 from antban/simplify-faulthandler-integration. Authored-by: antban Signed-off-by: Takuya Ueshin (cherry picked from commit 6227fbab03082a19ad04ab300b89a721334eb9bb) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/worker/analyze_udtf.py | 27 +++----- .../sql/worker/commit_data_source_write.py | 27 +++----- .../pyspark/sql/worker/create_data_source.py | 27 +++----- .../worker/data_source_pushdown_filters.py | 27 +++----- .../pyspark/sql/worker/lookup_data_sources.py | 27 +++----- .../sql/worker/plan_data_source_read.py | 27 +++----- .../worker/python_streaming_sink_runner.py | 27 +++----- .../sql/worker/write_into_data_source.py | 27 +++----- python/pyspark/util.py | 66 +++++++++++++++++++ python/pyspark/worker.py | 29 +++----- 10 files changed, 138 insertions(+), 173 deletions(-) diff --git a/python/pyspark/sql/worker/analyze_udtf.py b/python/pyspark/sql/worker/analyze_udtf.py index 665b1297fbc1f..526cb316862c7 100644 --- a/python/pyspark/sql/worker/analyze_udtf.py +++ b/python/pyspark/sql/worker/analyze_udtf.py @@ -15,7 +15,6 @@ # limitations under the License. # -import faulthandler import inspect import os import sys @@ -35,7 +34,12 @@ from pyspark.sql.functions import OrderingColumn, PartitioningColumn, SelectedColumn from pyspark.sql.types import _parse_datatype_json_string, StructType from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -100,6 +104,7 @@ def read_arguments(infile: IO) -> Tuple[List[AnalyzeArgument], Dict[str, Analyze return args, kwargs +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Runs the Python UDTF's `analyze` static method. @@ -108,18 +113,10 @@ def main(infile: IO, outfile: IO) -> None: in JVM and receive the Python UDTF and its arguments for the `analyze` static method, and call the `analyze` static method, and send back a AnalyzeResult as a result of the method. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -266,11 +263,6 @@ def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySpar except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -282,9 +274,6 @@ def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySpar write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/commit_data_source_write.py b/python/pyspark/sql/worker/commit_data_source_write.py index fb82b65f31229..37fee6ad8357e 100644 --- a/python/pyspark/sql/worker/commit_data_source_write.py +++ b/python/pyspark/sql/worker/commit_data_source_write.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler import os import sys from typing import IO @@ -29,7 +28,12 @@ SpecialLengths, ) from pyspark.sql.datasource import DataSourceWriter, WriterCommitMessage -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, pickleSer, @@ -40,6 +44,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for committing or aborting a data source write operation. @@ -49,18 +54,10 @@ def main(infile: IO, outfile: IO) -> None: responsible for invoking either the `commit` or the `abort` method on a data source writer instance, given a list of commit messages. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -106,11 +103,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -122,9 +114,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/create_data_source.py b/python/pyspark/sql/worker/create_data_source.py index 15e8fdc618e29..bf6ceda41ffb9 100644 --- a/python/pyspark/sql/worker/create_data_source.py +++ b/python/pyspark/sql/worker/create_data_source.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler import inspect import os import sys @@ -32,7 +31,12 @@ ) from pyspark.sql.datasource import DataSource, CaseInsensitiveDict from pyspark.sql.types import _parse_datatype_json_string, StructType -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -45,6 +49,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for creating a Python data source instance. @@ -62,18 +67,10 @@ def main(infile: IO, outfile: IO) -> None: This process then creates a `DataSource` instance using the above information and sends the pickled instance as well as the schema back to the JVM. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -172,11 +169,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -188,9 +180,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/data_source_pushdown_filters.py b/python/pyspark/sql/worker/data_source_pushdown_filters.py index 8601521bcfb13..7d255e1dbf778 100644 --- a/python/pyspark/sql/worker/data_source_pushdown_filters.py +++ b/python/pyspark/sql/worker/data_source_pushdown_filters.py @@ -16,7 +16,6 @@ # import base64 -import faulthandler import json import os import sys @@ -49,7 +48,12 @@ ) from pyspark.sql.types import StructType, VariantVal, _parse_datatype_json_string from pyspark.sql.worker.plan_data_source_read import write_read_func_and_partitions -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, pickleSer, @@ -119,6 +123,7 @@ def deserializeFilter(jsonDict: dict) -> Filter: return filter +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for planning a data source read with filter pushdown. @@ -140,18 +145,10 @@ def main(infile: IO, outfile: IO) -> None: on the reader and determines which filters are supported. The indices of the supported filters are sent back to the JVM, along with the list of partitions and the read function. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -258,11 +255,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -274,9 +266,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/lookup_data_sources.py b/python/pyspark/sql/worker/lookup_data_sources.py index eeb84263d4452..b23903cac8cb8 100644 --- a/python/pyspark/sql/worker/lookup_data_sources.py +++ b/python/pyspark/sql/worker/lookup_data_sources.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler from importlib import import_module from pkgutil import iter_modules import os @@ -29,7 +28,12 @@ SpecialLengths, ) from pyspark.sql.datasource import DataSource -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, pickleSer, @@ -40,6 +44,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for looking up the available Python Data Sources in Python path. @@ -51,18 +56,10 @@ def main(infile: IO, outfile: IO) -> None: This is responsible for searching the available Python Data Sources so they can be statically registered automatically. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -89,11 +86,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -105,9 +97,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/plan_data_source_read.py b/python/pyspark/sql/worker/plan_data_source_read.py index db79e58f2ec4f..51036f17586f2 100644 --- a/python/pyspark/sql/worker/plan_data_source_read.py +++ b/python/pyspark/sql/worker/plan_data_source_read.py @@ -15,7 +15,6 @@ # limitations under the License. # -import faulthandler import os import sys import functools @@ -47,7 +46,12 @@ BinaryType, StructType, ) -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -267,6 +271,7 @@ def data_source_read_func(iterator: Iterable[pa.RecordBatch]) -> Iterable[pa.Rec write_int(0, outfile) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for planning a data source read. @@ -287,18 +292,10 @@ def main(infile: IO, outfile: IO) -> None: The partition values and the Arrow Batch are then serialized and sent back to the JVM via the socket. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -402,11 +399,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -418,9 +410,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/python_streaming_sink_runner.py b/python/pyspark/sql/worker/python_streaming_sink_runner.py index ed6907ce5b63d..5ca3307fca33c 100644 --- a/python/pyspark/sql/worker/python_streaming_sink_runner.py +++ b/python/pyspark/sql/worker/python_streaming_sink_runner.py @@ -15,7 +15,6 @@ # limitations under the License. # -import faulthandler import os import sys from typing import IO @@ -35,7 +34,12 @@ _parse_datatype_json_string, StructType, ) -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -48,6 +52,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for committing or aborting a data source streaming write operation. @@ -57,18 +62,10 @@ def main(infile: IO, outfile: IO) -> None: responsible for invoking either the `commit` or the `abort` method on a data source writer instance, given a list of commit messages. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() setup_spark_files(infile) setup_broadcasts(infile) @@ -138,11 +135,6 @@ def main(infile: IO, outfile: IO) -> None: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -154,9 +146,6 @@ def main(infile: IO, outfile: IO) -> None: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index 917d0ca8e0079..b8a54f8397dc8 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import faulthandler import inspect import os import sys @@ -46,7 +45,12 @@ BinaryType, _create_row, ) -from pyspark.util import handle_worker_exception, local_connect_and_auth +from pyspark.util import ( + handle_worker_exception, + local_connect_and_auth, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark.worker_util import ( check_python_version, read_command, @@ -59,6 +63,7 @@ ) +@with_faulthandler def main(infile: IO, outfile: IO) -> None: """ Main method for saving into a Python data source. @@ -78,18 +83,10 @@ def main(infile: IO, outfile: IO) -> None: instance and send a function using the writer instance that can be used in mapInPandas/mapInArrow back to the JVM. """ - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - check_python_version(infile) - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) + start_faulthandler_periodic_traceback() memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) @@ -264,11 +261,6 @@ def batch_to_rows() -> Iterator[Row]: except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) send_accumulator_updates(outfile) @@ -280,9 +272,6 @@ def batch_to_rows() -> Iterator[Row]: write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 501d0afbb163f..9935206df1771 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -18,6 +18,7 @@ import copy import functools +import faulthandler import itertools import os import platform @@ -917,6 +918,71 @@ def default_api_mode() -> str: return "classic" +class _FaulthandlerHelper: + def __init__(self) -> None: + self._log_path: Optional[str] = None + self._log_file: Optional[TextIO] = None + self._periodic_traceback = False + + def start(self) -> None: + if self._log_path: + raise Exception("Fault handler is already registered. No second registration allowed") + self._log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) + if self._log_path: + self._log_path = os.path.join(self._log_path, str(os.getpid())) + self._log_file = open(self._log_path, "w") + + faulthandler.enable(file=self._log_file) + + def stop(self) -> None: + if self._log_path: + faulthandler.disable() + if self._log_file: + self._log_file.close() + self._log_file = None + try: + os.remove(self._log_path) + finally: + self._log_path = None + if self._periodic_traceback: + faulthandler.cancel_dump_traceback_later() + self._periodic_traceback = False + + def start_periodic_traceback(self) -> None: + # If the registration is already done - do nothing + if self._periodic_traceback: + return + + traceback_dump_interval_seconds = os.environ.get( + "PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None + ) + if traceback_dump_interval_seconds is not None and int(traceback_dump_interval_seconds) > 0: + self._periodic_traceback = True + faulthandler.dump_traceback_later(int(traceback_dump_interval_seconds), repeat=True) + + def with_faulthandler(self, func: Callable) -> Callable: + """ + Registers fault handler for the duration of function execution. + After function execution is over the faulthandler registration is cleaned as well, + including any files created for the integration. + """ + + @functools.wraps(func) + def wrapper(*args: Any, **kwargs: Any) -> Any: + try: + self.start() + return func(*args, **kwargs) + finally: + self.stop() + + return wrapper + + +_faulthandler_helper = _FaulthandlerHelper() +with_faulthandler = _faulthandler_helper.with_faulthandler +start_faulthandler_periodic_traceback = _faulthandler_helper.start_periodic_traceback + + if __name__ == "__main__": if "pypy" not in platform.python_implementation().lower(): import doctest diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 6e34b041665ac..4bae9f6dc48f5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -27,7 +27,6 @@ import itertools import json from typing import Any, Callable, Iterable, Iterator, Optional, Tuple -import faulthandler from pyspark.accumulators import ( SpecialAccumulatorIds, @@ -84,7 +83,12 @@ _create_row, _parse_datatype_json_string, ) -from pyspark.util import fail_on_stopiteration, handle_worker_exception +from pyspark.util import ( + fail_on_stopiteration, + handle_worker_exception, + with_faulthandler, + start_faulthandler_periodic_traceback, +) from pyspark import shuffle from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError from pyspark.worker_util import ( @@ -3290,23 +3294,14 @@ def func(_, it): return func, None, ser, ser +@with_faulthandler def main(infile, outfile): - faulthandler_log_path = os.environ.get("PYTHON_FAULTHANDLER_DIR", None) - tracebackDumpIntervalSeconds = os.environ.get("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", None) try: - if faulthandler_log_path: - faulthandler_log_path = os.path.join(faulthandler_log_path, str(os.getpid())) - faulthandler_log_file = open(faulthandler_log_path, "w") - faulthandler.enable(file=faulthandler_log_file) - boot_time = time.time() split_index = read_int(infile) if split_index == -1: # for unit tests sys.exit(-1) - - if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0: - faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True) - + start_faulthandler_periodic_traceback() check_python_version(infile) # read inputs only for a barrier task @@ -3397,11 +3392,6 @@ def process(): except BaseException as e: handle_worker_exception(e, outfile) sys.exit(-1) - finally: - if faulthandler_log_path: - faulthandler.disable() - faulthandler_log_file.close() - os.remove(faulthandler_log_path) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) write_long(shuffle.MemoryBytesSpilled, outfile) @@ -3419,9 +3409,6 @@ def process(): write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) sys.exit(-1) - # Force to cancel dump_traceback_later - faulthandler.cancel_dump_traceback_later() - if __name__ == "__main__": # Read information about how to connect back to the JVM from the environment. From 50f4adf1a4ddd27a2e44eb8d4111b9974c8077da Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Fri, 21 Nov 2025 20:05:19 -0800 Subject: [PATCH 194/400] [SPARK-54289][SQL] Allow MERGE INTO to preserve existing struct fields for UPDATE SET * when source struct has less nested fields than target struct ### What changes were proposed in this pull request? Introduce a new flag spark.sql.merge.nested.type.assign.by.field that allows UPDATE SET * action in MERGE INTO to be shorthand to assign every nested struct to its existing source counterpart (ie, UPDATE SET a.b.c = source.a.b.c). This will have the implication that existing struct field in the target table that has no source equivalent are preserved, when the corresponding source struct has less fields than target. Additional code is added to prevent null expansion in this case (ie, a null source struct expanding to a struct of nulls). ### Why are the changes needed? Following https://github.com/apache/spark/pull/52347, we now allow MERGE INTO to have a source table struct with less nested fields than target table struct. In this scenario, a user making a UPDATE SET * may have two interpretations. The use may interpret UPDATE SET * as shorthand to assign every top-column level field, ie UPDATE SET struct=source.struct, then the target struct is set to source struct object as is, with missing fields as NULL. This is the current behavior. The user may also mean that UPDATE SET * is short-hand to assign every nested struct field (ie, UPDATE SET struct.a.b = source.struct.a.b), in which case the target struct fields missing in source are retained. This is similar to UPDATE SET * not overriding existing target columns missing in the source, for example. For this case, this flag is added. ### Does this PR introduce _any_ user-facing change? No, the support to allow source structs to have less fields than target structs in MERGE INTO is unreleased yet (https://github.com/apache/spark/pull/52347), and in any case there is a flag to toggle this functionality. ### How was this patch tested? Unit tests, especially around cases where the source struct is null. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53149 from szehon-ho/merge_schema_evolution_update_nested. Authored-by: Szehon Ho Signed-off-by: Dongjoon Hyun (cherry picked from commit 966e0539c67b8b5ce9219c0bab60656e6ed1364b) Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 13 +- .../catalyst/analysis/AssignmentUtils.scala | 195 +++- .../ResolveRowLevelCommandAssignments.scala | 11 +- .../analysis/RewriteMergeIntoTable.scala | 10 +- .../ReplaceNullWithFalseInPredicate.scala | 3 +- .../catalyst/plans/logical/v2Commands.scala | 23 +- .../sql/catalyst/types/DataTypeUtils.scala | 33 + .../apache/spark/sql/internal/SQLConf.scala | 23 +- .../connector/MergeIntoTableSuiteBase.scala | 903 +++++++++++++++--- .../command/AlignMergeAssignmentsSuite.scala | 28 +- .../command/PlanResolutionSuite.scala | 40 +- 11 files changed, 1075 insertions(+), 207 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6b0665c1b7f35..fb8a84a85fc89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1709,14 +1709,15 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val resolvedDeleteCondition = deleteCondition.map( resolveExpressionByPlanChildren(_, m)) DeleteAction(resolvedDeleteCondition) - case UpdateAction(updateCondition, assignments) => + case UpdateAction(updateCondition, assignments, fromStar) => val resolvedUpdateCondition = updateCondition.map( resolveExpressionByPlanChildren(_, m)) UpdateAction( resolvedUpdateCondition, // The update value can access columns from both target and source tables. resolveAssignments(assignments, m, MergeResolvePolicy.BOTH, - throws = throws)) + throws = throws), + fromStar) case UpdateStarAction(updateCondition) => // Expand star to top level source columns. If source has less columns than target, // assignments will be added by ResolveRowLevelCommandAssignments later. @@ -1738,7 +1739,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor updateCondition.map(resolveExpressionByPlanChildren(_, m)), // For UPDATE *, the value must be from source table. resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE, - throws = throws)) + throws = throws), + fromStar = true) case o => o } val newNotMatchedActions = m.notMatchedActions.map { @@ -1783,14 +1785,15 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val resolvedDeleteCondition = deleteCondition.map( resolveExpressionByPlanOutput(_, targetTable)) DeleteAction(resolvedDeleteCondition) - case UpdateAction(updateCondition, assignments) => + case UpdateAction(updateCondition, assignments, fromStar) => val resolvedUpdateCondition = updateCondition.map( resolveExpressionByPlanOutput(_, targetTable)) UpdateAction( resolvedUpdateCondition, // The update value can access columns from the target table only. resolveAssignments(assignments, m, MergeResolvePolicy.TARGET, - throws = throws)) + throws = throws), + fromStar) case o => o } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala index 145c9077a4c2b..6cbc17c673817 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala @@ -21,13 +21,15 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.TableOutputResolver.DefaultValueFillMode.{NONE, RECURSE} -import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, Expression, GetStructField, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, CreateNamedStruct, Expression, GetStructField, If, IsNull, Literal} +import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull import org.apache.spark.sql.catalyst.plans.logical.Assignment import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getDefaultValueExprOrNullLit import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -50,13 +52,18 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { * * @param attrs table attributes * @param assignments assignments to align + * @param fromStar whether the assignments were resolved from an UPDATE SET * clause. + * These updates may assign struct fields individually + * (preserving existing fields). * @param coerceNestedTypes whether to coerce nested types to match the target type * for complex types + * @param missingSourcePaths paths that exist in target but not in source * @return aligned update assignments that match table attributes */ def alignUpdateAssignments( attrs: Seq[Attribute], assignments: Seq[Assignment], + fromStar: Boolean, coerceNestedTypes: Boolean): Seq[Assignment] = { val errors = new mutable.ArrayBuffer[String]() @@ -68,7 +75,8 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { assignments, addError = err => errors += err, colPath = Seq(attr.name), - coerceNestedTypes) + coerceNestedTypes, + fromStar) } if (errors.nonEmpty) { @@ -152,7 +160,8 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { assignments: Seq[Assignment], addError: String => Unit, colPath: Seq[String], - coerceNestedTypes: Boolean = false): Expression = { + coerceNestedTypes: Boolean = false, + updateStar: Boolean = false): Expression = { val (exactAssignments, otherAssignments) = assignments.partition { assignment => assignment.key.semanticEquals(colExpr) @@ -174,9 +183,31 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { } else if (exactAssignments.isEmpty && fieldAssignments.isEmpty) { TableOutputResolver.checkNullability(colExpr, col, conf, colPath) } else if (exactAssignments.nonEmpty) { - val value = exactAssignments.head.value - val coerceMode = if (coerceNestedTypes) RECURSE else NONE - TableOutputResolver.resolveUpdate("", value, col, conf, addError, colPath, coerceMode) + if (SQLConf.get.mergeUpdateStructsByField && updateStar) { + val value = exactAssignments.head.value + col.dataType match { + case structType: StructType => + // Expand assignments to leaf fields + val structAssignment = + applyNestedFieldAssignments(col, colExpr, value, addError, colPath, + coerceNestedTypes) + + // Wrap with null check for missing source fields + fixNullExpansion(col, value, structType, structAssignment, + colPath, addError) + case _ => + // For non-struct types, resolve directly + val coerceMode = if (coerceNestedTypes) RECURSE else NONE + TableOutputResolver.resolveUpdate("", value, col, conf, addError, colPath, + coerceMode) + } + } else { + val value = exactAssignments.head.value + val coerceMode = if (coerceNestedTypes) RECURSE else NONE + val resolvedValue = TableOutputResolver.resolveUpdate("", value, col, conf, addError, + colPath, coerceMode) + resolvedValue + } } else { applyFieldAssignments(col, colExpr, fieldAssignments, addError, colPath, coerceNestedTypes) } @@ -210,6 +241,63 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { } } + private def applyNestedFieldAssignments( + col: Attribute, + colExpr: Expression, + value: Expression, + addError: String => Unit, + colPath: Seq[String], + coerceNestedTyptes: Boolean): Expression = { + + col.dataType match { + case structType: StructType => + val fieldAttrs = DataTypeUtils.toAttributes(structType) + + val updatedFieldExprs = fieldAttrs.zipWithIndex.map { case (fieldAttr, ordinal) => + val fieldPath = colPath :+ fieldAttr.name + val targetFieldExpr = GetStructField(colExpr, ordinal, Some(fieldAttr.name)) + + // Try to find a corresponding field in the source value by name + val sourceFieldValue: Expression = value.dataType match { + case valueStructType: StructType => + valueStructType.fields.find(f => conf.resolver(f.name, fieldAttr.name)) match { + case Some(matchingField) => + // Found matching field in source, extract it + val fieldIndex = valueStructType.fieldIndex(matchingField.name) + GetStructField(value, fieldIndex, Some(matchingField.name)) + case None => + // Field doesn't exist in source, use target's current value with null check + TableOutputResolver.checkNullability(targetFieldExpr, fieldAttr, conf, fieldPath) + } + case _ => + // Value is not a struct, cannot extract field + addError(s"Cannot assign non-struct value to struct field '${fieldPath.quoted}'") + Literal(null, fieldAttr.dataType) + } + + // Recurse or resolve based on field type + fieldAttr.dataType match { + case nestedStructType: StructType => + // Field is a struct, recurse + applyNestedFieldAssignments(fieldAttr, targetFieldExpr, sourceFieldValue, + addError, fieldPath, coerceNestedTyptes) + case _ => + // Field is not a struct, resolve with TableOutputResolver + val coerceMode = if (coerceNestedTyptes) RECURSE else NONE + TableOutputResolver.resolveUpdate("", sourceFieldValue, fieldAttr, conf, addError, + fieldPath, coerceMode) + } + } + toNamedStruct(structType, updatedFieldExprs) + + case otherType => + addError( + "Updating nested fields is only supported for StructType but " + + s"'${colPath.quoted}' is of type $otherType") + colExpr + } + } + private def toNamedStruct(structType: StructType, fieldExprs: Seq[Expression]): Expression = { val namedStructExprs = structType.fields.zip(fieldExprs).flatMap { case (field, expr) => Seq(Literal(field.name), expr) @@ -217,6 +305,101 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { CreateNamedStruct(namedStructExprs) } + private def getMissingSourcePaths(targetType: StructType, + sourceType: DataType, + colPath: Seq[String], + addError: String => Unit): Seq[Seq[String]] = { + val nestedTargetPaths = DataTypeUtils.extractLeafFieldPaths(targetType, Seq.empty) + val nestedSourcePaths = sourceType match { + case sourceStructType: StructType => + DataTypeUtils.extractLeafFieldPaths(sourceStructType, Seq.empty) + case _ => + addError(s"Value for struct type: " + + s"${colPath.quoted} must be a struct but was ${sourceType.simpleString}") + Seq() + } + nestedSourcePaths.diff(nestedTargetPaths) + } + + /** + * Creates a null check for a field at the given path within a struct expression. + * Navigates through the struct hierarchy following the path and returns an IsNull check + * for the final field. + * + * @param rootExpr the root expression to navigate from + * @param path the field path to navigate (sequence of field names) + * @return an IsNull expression checking if the field at the path is null + */ + private def createNullCheckForFieldPath( + rootExpr: Expression, + path: Seq[String]): Expression = { + var currentExpr: Expression = rootExpr + path.foreach { fieldName => + currentExpr.dataType match { + case st: StructType => + st.fields.find(f => conf.resolver(f.name, fieldName)) match { + case Some(field) => + val fieldIndex = st.fieldIndex(field.name) + currentExpr = GetStructField(currentExpr, fieldIndex, Some(field.name)) + case None => + // Field not found, shouldn't happen + } + case _ => + // Not a struct, shouldn't happen + } + } + IsNull(currentExpr) + } + + /** + * As UPDATE SET * can assign struct fields individually (preserving existing fields), + * this will lead to null expansion, ie, a struct is created where all fields are null. + * Wraps a struct assignment with null checks for the source and missing source fields. + * Return null if all are null. + * + * @param col the target column attribute + * @param value the source value expression + * @param structType the target struct type + * @param structAssignment the struct assignment result to wrap + * @param colPath the column path for error reporting + * @param addError error reporting function + * @return the wrapped expression with null checks + */ + private def fixNullExpansion( + col: Attribute, + value: Expression, + structType: StructType, + structAssignment: Expression, + colPath: Seq[String], + addError: String => Unit): Expression = { + // As StoreAssignmentPolicy.LEGACY is not allowed in DSv2, always add null check for + // non-nullable column + if (!col.nullable) { + AssertNotNull(value) + } else { + // Check if source struct is null + val valueIsNull = IsNull(value) + + // Check if missing source paths (paths in target but not in source) are not null + // These will be null for the case of UPDATE SET * and + val missingSourcePaths = getMissingSourcePaths(structType, value.dataType, colPath, addError) + val condition = if (missingSourcePaths.nonEmpty) { + // Check if all target attributes at missing source paths are null + val missingFieldNullChecks = missingSourcePaths.map { path => + createNullCheckForFieldPath(col, path) + } + // Combine all null checks with AND + val allMissingFieldsNull = missingFieldNullChecks.reduce[Expression]((a, b) => And(a, b)) + And(valueIsNull, allMissingFieldsNull) + } else { + valueIsNull + } + + // Return: If (condition) THEN NULL ELSE structAssignment + If(condition, Literal(null, structAssignment.dataType), structAssignment) + } + } + /** * Checks whether assignments are aligned and compatible with table columns. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala index 3eb528954b352..93ef98e3183af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala @@ -44,7 +44,7 @@ object ResolveRowLevelCommandAssignments extends Rule[LogicalPlan] { validateStoreAssignmentPolicy() val newTable = cleanAttrMetadata(u.table) val newAssignments = AssignmentUtils.alignUpdateAssignments(u.table.output, u.assignments, - coerceNestedTypes = false) + fromStar = false, coerceNestedTypes = false) u.copy(table = newTable, assignments = newAssignments) case u: UpdateTable if !u.skipSchemaResolution && u.resolved && !u.aligned => @@ -53,10 +53,11 @@ object ResolveRowLevelCommandAssignments extends Rule[LogicalPlan] { case m: MergeIntoTable if !m.skipSchemaResolution && m.resolved && m.rewritable && !m.aligned && !m.needSchemaEvolution => validateStoreAssignmentPolicy() - val coerceNestedTypes = SQLConf.get.coerceMergeNestedTypes + val coerceNestedTypes = SQLConf.get.mergeCoerceNestedTypes m.copy( targetTable = cleanAttrMetadata(m.targetTable), - matchedActions = alignActions(m.targetTable.output, m.matchedActions, coerceNestedTypes), + matchedActions = alignActions(m.targetTable.output, m.matchedActions, + coerceNestedTypes), notMatchedActions = alignActions(m.targetTable.output, m.notMatchedActions, coerceNestedTypes), notMatchedBySourceActions = alignActions(m.targetTable.output, m.notMatchedBySourceActions, @@ -117,9 +118,9 @@ object ResolveRowLevelCommandAssignments extends Rule[LogicalPlan] { actions: Seq[MergeAction], coerceNestedTypes: Boolean): Seq[MergeAction] = { actions.map { - case u @ UpdateAction(_, assignments) => + case u @ UpdateAction(_, assignments, fromStar) => u.copy(assignments = AssignmentUtils.alignUpdateAssignments(attrs, assignments, - coerceNestedTypes)) + fromStar, coerceNestedTypes)) case d: DeleteAction => d case i @ InsertAction(_, assignments) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala index 8b5b690aa7403..1d2e2fef20965 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala @@ -334,7 +334,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper // original row ID values must be preserved and passed back to the table to encode updates // if there are any assignments to row ID attributes, add extra columns for original values val updateAssignments = (matchedActions ++ notMatchedBySourceActions).flatMap { - case UpdateAction(_, assignments) => assignments + case UpdateAction(_, assignments, _) => assignments case _ => Nil } buildOriginalRowIdValues(rowIdAttrs, updateAssignments) @@ -434,7 +434,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper // converts a MERGE action into an instruction on top of the joined plan for group-based plans private def toInstruction(action: MergeAction, metadataAttrs: Seq[Attribute]): Instruction = { action match { - case UpdateAction(cond, assignments) => + case UpdateAction(cond, assignments, _) => val rowValues = assignments.map(_.value) val metadataValues = nullifyMetadataOnUpdate(metadataAttrs) val output = Seq(Literal(WRITE_WITH_METADATA_OPERATION)) ++ rowValues ++ metadataValues @@ -466,12 +466,12 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper splitUpdates: Boolean): Instruction = { action match { - case UpdateAction(cond, assignments) if splitUpdates => + case UpdateAction(cond, assignments, _) if splitUpdates => val output = deltaDeleteOutput(rowAttrs, rowIdAttrs, metadataAttrs, originalRowIdValues) val otherOutput = deltaReinsertOutput(assignments, metadataAttrs, originalRowIdValues) Split(cond.getOrElse(TrueLiteral), output, otherOutput) - case UpdateAction(cond, assignments) => + case UpdateAction(cond, assignments, _) => val output = deltaUpdateOutput(assignments, metadataAttrs, originalRowIdValues) Keep(Update, cond.getOrElse(TrueLiteral), output) @@ -495,7 +495,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper val actions = merge.matchedActions ++ merge.notMatchedActions ++ merge.notMatchedBySourceActions actions.foreach { case DeleteAction(Some(cond)) => checkMergeIntoCondition("DELETE", cond) - case UpdateAction(Some(cond), _) => checkMergeIntoCondition("UPDATE", cond) + case UpdateAction(Some(cond), _, _) => checkMergeIntoCondition("UPDATE", cond) case InsertAction(Some(cond), _) => checkMergeIntoCondition("INSERT", cond) case _ => // OK } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala index 7134c3daf3baa..9a676571d1071 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala @@ -149,7 +149,8 @@ object ReplaceNullWithFalseInPredicate extends Rule[LogicalPlan] { private def replaceNullWithFalse(mergeActions: Seq[MergeAction]): Seq[MergeAction] = { mergeActions.map { - case u @ UpdateAction(Some(cond), _) => u.copy(condition = Some(replaceNullWithFalse(cond))) + case u @ UpdateAction(Some(cond), _, _) => + u.copy(condition = Some(replaceNullWithFalse(cond))) case u @ UpdateStarAction(Some(cond)) => u.copy(condition = Some(replaceNullWithFalse(cond))) case d @ DeleteAction(Some(cond)) => d.copy(condition = Some(replaceNullWithFalse(cond))) case i @ InsertAction(Some(cond), _) => i.copy(condition = Some(replaceNullWithFalse(cond))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index dcce220402449..26ce138523e7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -873,7 +873,7 @@ case class MergeIntoTable( lazy val aligned: Boolean = { val actions = matchedActions ++ notMatchedActions ++ notMatchedBySourceActions actions.forall { - case UpdateAction(_, assignments) => + case UpdateAction(_, assignments, _) => AssignmentUtils.aligned(targetTable.output, assignments) case _: DeleteAction => true @@ -926,10 +926,7 @@ case class MergeIntoTable( case a: UpdateAction => a.assignments case a: InsertAction => a.assignments }.flatten - - val sourcePaths = MergeIntoTable.extractAllFieldPaths(sourceTable.schema) - // Only allow unresolved assignment keys to be candidates for schema evolution - // if they are directly assigned from source fields, ie UPDATE SET new = source.new + val sourcePaths = DataTypeUtils.extractAllFieldPaths(sourceTable.schema) assignments.forall { assignment => assignment.resolved || (assignment.value.resolved && sourcePaths.exists { @@ -1083,19 +1080,6 @@ object MergeIntoTable { filterSchema(merge.sourceTable.schema, Seq.empty) } - private def extractAllFieldPaths(schema: StructType, basePath: Seq[String] = Seq.empty): - Seq[Seq[String]] = { - schema.flatMap { field => - val fieldPath = basePath :+ field.name - field.dataType match { - case struct: StructType => - fieldPath +: extractAllFieldPaths(struct, fieldPath) - case _ => - Seq(fieldPath) - } - } - } - // Helper method to extract field path from an Expression. private def extractFieldPath(expr: Expression, allowUnresolved: Boolean): Seq[String] = { expr match { @@ -1142,7 +1126,8 @@ case class DeleteAction(condition: Option[Expression]) extends MergeAction { case class UpdateAction( condition: Option[Expression], - assignments: Seq[Assignment]) extends MergeAction { + assignments: Seq[Assignment], + fromStar: Boolean = false) extends MergeAction { override def children: Seq[Expression] = condition.toSeq ++ assignments override protected def withNewChildrenInternal( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala index c6e51aab4584b..e7bd5bd1aa2d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala @@ -249,5 +249,38 @@ object DataTypeUtils { case v: Long => fromDecimal(Decimal(BigDecimal(v))) case _ => forType(literal.dataType) } + + /** + * Extracts all struct field paths from a nested StructType. + */ + def extractAllFieldPaths(schema: StructType, basePath: Seq[String] = Seq.empty): + Seq[Seq[String]] = { + schema.flatMap { field => + val fieldPath = basePath :+ field.name + field.dataType match { + case struct: StructType => + fieldPath +: extractAllFieldPaths(struct, fieldPath) + case _ => + Seq(fieldPath) + } + } + } + + /** + * Extracts only leaf-level field paths from a nested StructType. + * Unlike extractAllFieldPaths, this method does not include intermediate struct paths. + */ + def extractLeafFieldPaths(schema: StructType, basePath: Seq[String] = Seq.empty): + Seq[Seq[String]] = { + schema.flatMap { field => + val fieldPath = basePath :+ field.name + field.dataType match { + case struct: StructType => + extractLeafFieldPaths(struct, fieldPath) + case _ => + Seq(fieldPath) + } + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5ae5fa33a721d..9633011cf872e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6611,8 +6611,8 @@ object SQLConf { .booleanConf .createWithDefault(true) - val MERGE_INTO_SOURCE_NESTED_TYPE_COERCION_ENABLED = - buildConf("spark.sql.merge.source.nested.type.coercion.enabled") + val MERGE_INTO_NESTED_TYPE_COERCION_ENABLED = + buildConf("spark.sql.merge.nested.type.coercion.enabled") .internal() .doc("If enabled, allow MERGE INTO to coerce source nested types if they have less" + "nested fields than the target table's nested types.") @@ -6620,6 +6620,18 @@ object SQLConf { .booleanConf .createWithDefault(true) + val MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD = + buildConf("spark.sql.merge.nested.type.assign.by.field") + .internal() + .doc("If enabled and spark.sql.merge.source.nested.type.coercion.enabled is true," + + "allow MERGE INTO with UPDATE SET * action to set nested structs field by field. " + + "In updated rows, target structs will preserve the original value for fields missing " + + "in the the source struct. If disabled, the entire target struct will be replaced, " + + "and fields missing in the source struct will be null.") + .version("4.1.0") + .booleanConf + .createWithDefault(true) + /** * Holds information about keys that have been deprecated. * @@ -7777,8 +7789,11 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def legacyXMLParserEnabled: Boolean = getConf(SQLConf.LEGACY_XML_PARSER_ENABLED) - def coerceMergeNestedTypes: Boolean = - getConf(SQLConf.MERGE_INTO_SOURCE_NESTED_TYPE_COERCION_ENABLED) + def mergeCoerceNestedTypes: Boolean = + getConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED) + + def mergeUpdateStructsByField: Boolean = + getConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD) /** ********************** SQLConf functionality methods ************ */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index 85b0faed4c388..7051a0b455e32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -3231,60 +3231,180 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase test("merge into schema evolution replace column with nested struct and set all columns") { Seq(true, false).foreach { withSchemaEvolution => - withTempView("source") { - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT, m: MAP>>, - |dep STRING""".stripMargin, - """{ "pk": 1, "s": { "c1": 2, "c2": { "a": [1,2], "m": { "a": "b" } } }, "dep": "hr" }""") + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { + withTempView("source") { + // Create table using Spark SQL + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - // missing column 'a' - StructField("m", MapType(StringType, StringType)), - StructField("c3", BooleanType) // new column - ))) - ))), - StructField("dep", StringType) - )) - val data = Seq( - Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), - Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") + // Insert data using DataFrame API with objects + val tableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) + .coalesce(1).writeTo(tableNameAsString).append() + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // missing column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + StructField("dep", StringType) + )) + val sourceData = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source") - val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" - val mergeStmt = - s"""MERGE $schemaEvolutionClause - |INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin - if (withSchemaEvolution) { - sql(mergeStmt) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "sales"), - Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) - } else { - val exception = intercept[org.apache.spark.sql.AnalysisException] { - sql(mergeStmt) + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + if (withSchemaEvolution) { + sql(mergeStmt) + if (updateByFields) { + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(Seq(1, 2), Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } else { + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) + } } - assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") - assert(exception.getMessage.contains( - "Cannot write extra fields `c3` to the struct `s`.`c2`")) + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + } + + test("merge into schema evolution replace column with nested struct and update " + + "top level struct") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { + withTempView("source") { + // Create table using Spark SQL + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + + // Insert data using DataFrame API with objects + val tableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) + .coalesce(1).writeTo(tableNameAsString).append() + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // missing column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + StructField("dep", StringType) + )) + val sourceData = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET s = src.s + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + if (withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "hr"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } @@ -4343,50 +4463,6 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } - test("merge into with source missing fields in top-level struct") { - withTempView("source") { - // Target table has struct with 3 fields at top level - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT, - |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": true }, "dep": "sales"}""") - - // Source table has struct with only 2 fields (c1, c2) - missing c3 - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StringType)))), // missing c3 field - StructField("dep", StringType))) - val data = Seq( - Row(1, Row(10, "b"), "hr"), - Row(2, Row(20, "c"), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") - - sql( - s"""MERGE INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) - - // Missing field c3 should be filled with NULL - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, "a", true), "sales"), - Row(1, Row(10, "b", null), "hr"), - Row(2, Row(20, "c", null), "engineering"))) - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") - } - test("merge into with source missing fields in struct nested in array") { withTempView("source") { // Target table has struct with 3 fields (c1, c2, c3) in array @@ -4540,22 +4616,459 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - test("merge into with source missing fields in nested struct") { - Seq(true, false).foreach { nestedTypeCoercion => - withSQLConf(SQLConf.MERGE_INTO_SOURCE_NESTED_TYPE_COERCION_ENABLED.key - -> nestedTypeCoercion.toString) { + test("merge into with source missing fields in top-level struct") { + withTempView("source") { + // Target table has struct with 3 fields at top level + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": true }, "dep": "sales"}""") + + // Source table has struct with only 2 fields (c1, c2) - missing c3 + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType)))), // missing c3 field + StructField("dep", StringType))) + val data = Seq( + Row(1, Row(10, "b"), "hr"), + Row(2, Row(20, "c"), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + + // Missing field c3 should be filled with NULL + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a", true), "sales"), + Row(1, Row(10, "b", null), "hr"), + Row(2, Row(20, "c", null), "engineering"))) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge with null struct") { + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) + + // Source table matches target table schema + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a"), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + + test("merge with null struct - update field") { + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) + + // Source table matches target table schema + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET s = source.s + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a"), "sales"), + Row(1, null, "hr"), + Row(2, null, "finance"))) + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + + test("merge with null struct into non-nullable struct column") { + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { withTempView("source") { - // Target table has nested struct: s.c1, s.c2.a, s.c2.b createAndInitTable( s"""pk INT NOT NULL, - |s STRUCT>, + |s STRUCT NOT NULL, |dep STRING""".stripMargin, - """{ "pk": 1, "s": { "c1": 2, "c2": { "a": 10, "b": true } } } - |{ "pk": 2, "s": { "c1": 2, "c2": { "a": 30, "b": false } } }""".stripMargin) + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) - // Source table is missing field 'b' in nested struct s.c2 + // Source table has null for the struct column val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + // Should throw an exception when trying to insert/update null into NOT NULL column + val exception = intercept[Exception] { + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + } + assert(exception.getMessage.contains( + "NULL value appeared in non-nullable field")) + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + + test("merge with with null struct with missing nested field") { + Seq(true, false).foreach { updateByFields => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf( + SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString, + SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target table has nested struct with fields c1 and c2 + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + .stripMargin) + + // Source table has null for the nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val mergeStmt = + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } else { + // Without coercion, the merge should fail due to missing field + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot find data for the output column `s`.`c2`.`b`.")) + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge null struct with schema evolution - source with missing and extra nested fields") { + Seq(true, false).foreach { updateByFields => + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf( + SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString, + SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target table has nested struct with fields c1 and c2 + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + .stripMargin) + + // Source table has missing field 'b' and extra field 'c' in nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + // missing field 'b' + StructField("c", StringType) // extra field 'c' + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes) { + if (withSchemaEvolution) { + // extra nested field is added + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x", null)), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } else { + // extra nested field is not added + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot write extra fields `c` to the struct `s`.`c2`")) + } + } else { + // Without source struct coercion, the merge should fail + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot find data for the output column `s`.`c2`.`b`.")) + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + } + + test("merge null struct with non-nullable nested field - source with missing " + + "and extra nested fields") { + + withSQLConf( + SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> "true", + SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> "true") { + withTempView("source") { + // Target table has nested struct with NON-NULLABLE field b + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + .stripMargin) + + // Source table has missing field 'b' and extra field 'c' in nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + // missing field 'b' (which is non-nullable in target) + StructField("c", StringType) // extra field 'c' + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val mergeStmt = + s"""MERGE WITH SCHEMA EVOLUTION + |INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + // All cases should fail due to non-nullable constraint violation + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains("Cannot write incompatible data for the table ``: " + + "Cannot find data for the output column `s`.`c2`.`b`.")) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + + test("merge with null struct using default value") { + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { + withTempView("source") { + // Target table has nested struct with a default value + sql( + s"""CREATE TABLE $tableNameAsString ( + | pk INT NOT NULL, + | s STRUCT> DEFAULT + | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), + | dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + + // Insert initial data using DataFrame API + val initialSchema = StructType(Seq( StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + val initialData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) + .writeTo(tableNameAsString).append() + + // Source table has null for the nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), StructField("s", StructType(Seq( StructField("c1", IntegerType), StructField("c2", StructType(Seq( @@ -4565,45 +5078,179 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase ))), StructField("dep", StringType) )) + val data = Seq( - Row(1, Row(10, Row(20)), "sales"), - Row(2, Row(20, Row(30)), "engineering") + Row(1, null, "engineering"), + Row(2, null, "finance") ) spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") - // Missing field b should be filled with NULL - val mergeStmt = s"""MERGE INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } - if (nestedTypeCoercion) { - sql(mergeStmt) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(20, null)), "sales"), - Row(2, Row(20, Row(30, null)), "engineering"))) - } else { - val exception = intercept[Exception] { + test("merge with source missing struct column with default value") { + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> + updateByFields.toString) { + withTempView("source") { + // Target table has nested struct with a default value + sql( + s"""CREATE TABLE $tableNameAsString ( + | pk INT NOT NULL, + | s STRUCT> DEFAULT + | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), + | dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + + // Insert initial data using DataFrame API + val initialSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + val initialData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) + .writeTo(tableNameAsString).append() + + // Source table is completely missing the struct column 's' + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, "engineering"), + Row(2, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + // When inserting without specifying the struct column, default should be used + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET dep = source.dep + |WHEN NOT MATCHED THEN + | INSERT (pk, dep) VALUES (source.pk, source.dep) + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "engineering"), + Row(2, Row(999, Row(999, "default")), "finance"))) + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + + test("merge into with source missing fields in nested struct") { + Seq(true, false).foreach { nestedTypeCoercion => + Seq(true, false).foreach { updateByFields => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key + -> updateByFields.toString, + SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key + -> nestedTypeCoercion.toString) { + withTempView("source") { + // Target table has nested struct: s.c1, s.c2.a, s.c2.b + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 1, "s": { "c1": 2, "c2": { "a": 10, "b": true } } } + |{ "pk": 2, "s": { "c1": 2, "c2": { "a": 30, "b": false } } }""".stripMargin) + + // Source table is missing field 'b' in nested struct s.c2 + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) + ))), + StructField("dep", StringType) + )) + val data = Seq( + Row(1, Row(10, Row(20)), "sales"), + Row(2, Row(20, Row(30)), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + // Missing field b should be filled with NULL + val mergeStmt = s"""MERGE INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (nestedTypeCoercion) { sql(mergeStmt) + if (updateByFields) { + // When updating by fields, only non-null fields are updated + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(20, true)), "sales"), + Row(2, Row(20, Row(30, false)), "engineering"))) + } else { + // When updating by top level column, the missing field is set to NULL + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(20, null)), "sales"), + Row(2, Row(20, Row(30, null)), "engineering"))) + } + } else { + val exception = intercept[Exception] { + sql(mergeStmt) + } + assert(exception.getMessage.contains( + """Cannot write incompatible data for the table ``""".stripMargin)) } - assert(exception.getMessage.contains( - """Cannot write incompatible data for the table ``""".stripMargin)) } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } } - - test("merge with named_struct missing non-nullable field backup") { + test("merge with named_struct missing non-nullable field") { withTempView("source") { createAndInitTable( s"""pk INT NOT NULL, @@ -4631,7 +5278,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase .createOrReplaceTempView("source") Seq(true, false).foreach { coerceNestedTypes => - withSQLConf(SQLConf.MERGE_INTO_SOURCE_NESTED_TYPE_COERCION_ENABLED.key -> + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> coerceNestedTypes.toString) { // Test UPDATE with named_struct missing non-nullable field c2 val e = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala index 8420e5e4d8808..f635131dc3f76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala @@ -40,7 +40,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { | UPDATE SET t.txt = "error", t.i = CAST(null AS INT)""".stripMargin) matchedActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, AssertNotNull(iValue: AttributeReference, _)), @@ -80,7 +80,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } notMatchedBySourceActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, AssertNotNull(_: Cast, _)), @@ -138,7 +138,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } matchedActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, iValue: AttributeReference), @@ -184,7 +184,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } notMatchedBySourceActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, iValue: AttributeReference), @@ -217,7 +217,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { |""".stripMargin) matchedActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, iValue: AttributeReference), @@ -280,7 +280,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } matchedActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, iValue: AttributeReference), @@ -342,7 +342,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } notMatchedBySourceActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(i: AttributeReference, iValue: AttributeReference), @@ -463,7 +463,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } matchedActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(c: AttributeReference, cValue: StaticInvoke), @@ -531,7 +531,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } notMatchedBySourceActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(c: AttributeReference, cValue: StaticInvoke), @@ -691,7 +691,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { assertNullCheckExists(plan4, Seq("s", "n_s", "dn_i")) Seq(true, false).foreach { coerceNestedTypes => - withSQLConf(SQLConf.MERGE_INTO_SOURCE_NESTED_TYPE_COERCION_ENABLED.key -> + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> coerceNestedTypes.toString) { val mergeStmt = s"""MERGE INTO nested_struct_table t USING nested_struct_table src @@ -745,7 +745,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { val actions = if (matchedActions.nonEmpty) matchedActions else notMatchedBySourceActions actions match { - case Seq(UpdateAction(_, assignments)) => + case Seq(UpdateAction(_, assignments, _)) => assignments match { case Seq( Assignment( @@ -858,7 +858,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { assertNullCheckExists(plan4, Seq("s", "n_s", "dn_i")) Seq(true, false).foreach { coerceNestedTypes => - withSQLConf(SQLConf.MERGE_INTO_SOURCE_NESTED_TYPE_COERCION_ENABLED.key -> + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> coerceNestedTypes.toString) { val mergeStmt = s"""MERGE INTO nested_struct_table t USING nested_struct_table src @@ -947,7 +947,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { | UPDATE SET t.i = DEFAULT""".stripMargin) matchedActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(b: AttributeReference, bValue: AttributeReference), @@ -1001,7 +1001,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } notMatchedBySourceActions match { - case Seq(UpdateAction(None, assignments)) => + case Seq(UpdateAction(None, assignments, _)) => assignments match { case Seq( Assignment(b: AttributeReference, bValue: AttributeReference), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index dfd24a1ebe970..fb56cecb05ae4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -1703,12 +1703,12 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), - updateAssigns)), + updateAssigns, _)), Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), insertAssigns)), Seq(DeleteAction(Some(EqualTo(ndl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(nul: AttributeReference, StringLiteral("update"))), - notMatchedBySourceUpdateAssigns)), + notMatchedBySourceUpdateAssigns, _)), withSchemaEvolution) => checkMergeConditionResolution(target, source, mergeCondition) checkMatchedClausesResolution(target, source, Some(dl), Some(ul), updateAssigns) @@ -1737,7 +1737,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, - StringLiteral("update"))), updateAssigns)), + StringLiteral("update"))), updateAssigns, _)), Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), insertAssigns)), Seq(), @@ -1766,7 +1766,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), SubqueryAlias(AliasIdentifier("source", Seq()), AsDataSourceV2Relation(source)), mergeCondition, - Seq(UpdateAction(None, updateAssigns)), + Seq(UpdateAction(None, updateAssigns, _)), Seq(InsertAction(None, insertAssigns)), Seq(), withSchemaEvolution) => @@ -1797,10 +1797,10 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), SubqueryAlias(AliasIdentifier("source", Seq()), AsDataSourceV2Relation(source)), mergeCondition, - Seq(DeleteAction(Some(_)), UpdateAction(None, updateAssigns)), + Seq(DeleteAction(Some(_)), UpdateAction(None, updateAssigns, _)), Seq(InsertAction(None, insertAssigns)), Seq(DeleteAction(Some(EqualTo(_: AttributeReference, StringLiteral("delete")))), - UpdateAction(None, notMatchedBySourceUpdateAssigns)), + UpdateAction(None, notMatchedBySourceUpdateAssigns, _)), withSchemaEvolution) => checkMergeConditionResolution(target, source, mergeCondition) checkMatchedClausesResolution(target, source, None, None, updateAssigns) @@ -1832,12 +1832,12 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), - updateAssigns)), + updateAssigns, _)), Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), insertAssigns)), Seq(DeleteAction(Some(EqualTo(ndl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(nul: AttributeReference, StringLiteral("update"))), - notMatchedBySourceUpdateAssigns)), + notMatchedBySourceUpdateAssigns, _)), withSchemaEvolution) => checkMergeConditionResolution(target, source, mergeCondition) checkMatchedClausesResolution(target, source, Some(dl), Some(ul), updateAssigns) @@ -1871,12 +1871,12 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), - updateAssigns)), + updateAssigns, _)), Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), insertAssigns)), Seq(DeleteAction(Some(EqualTo(ndl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(nul: AttributeReference, StringLiteral("update"))), - notMatchedBySourceUpdateAssigns)), + notMatchedBySourceUpdateAssigns, _)), withSchemaEvolution) => checkMergeConditionResolution(target, source, mergeCondition) checkMatchedClausesResolution(target, source, Some(dl), Some(ul), updateAssigns) @@ -1927,7 +1927,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { case UpdateAction(Some(EqualTo(_: AttributeReference, StringLiteral("update"))), Seq( Assignment(_: AttributeReference, Literal(null, StringType)), - Assignment(_: AttributeReference, _: AttributeReference))) => + Assignment(_: AttributeReference, _: AttributeReference)), _) => case other => fail("unexpected second matched action " + other) } assert(m.notMatchedActions.length == 1) @@ -1947,7 +1947,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { } m.notMatchedBySourceActions(1) match { case UpdateAction(Some(EqualTo(_: AttributeReference, StringLiteral("update"))), - Seq(Assignment(_: AttributeReference, Literal(null, StringType)))) => + Seq(Assignment(_: AttributeReference, Literal(null, StringType))), _) => case other => fail("unexpected second not matched by source action " + other) } @@ -1999,7 +1999,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { val second = m.matchedActions(1) second match { case UpdateAction(Some(EqualTo(_: AttributeReference, Literal(31, IntegerType))), - Seq(Assignment(_: AttributeReference, Literal(42, IntegerType)))) => + Seq(Assignment(_: AttributeReference, Literal(42, IntegerType))), _) => case other => fail("unexpected second matched action " + other) } assert(m.notMatchedActions.length == 1) @@ -2017,7 +2017,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { } m.notMatchedBySourceActions(1) match { case UpdateAction(Some(EqualTo(_: AttributeReference, Literal(31, IntegerType))), - Seq(Assignment(_: AttributeReference, Literal(42, IntegerType)))) => + Seq(Assignment(_: AttributeReference, Literal(42, IntegerType))), _) => case other => fail("unexpected second not matched by source action " + other) } assert(m.withSchemaEvolution === false) @@ -2158,11 +2158,11 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { AsDataSourceV2Relation(target), AsDataSourceV2Relation(source), _, - Seq(DeleteAction(Some(_)), UpdateAction(None, firstUpdateAssigns)), + Seq(DeleteAction(Some(_)), UpdateAction(None, firstUpdateAssigns, _)), Seq(InsertAction( Some(EqualTo(il: AttributeReference, StringLiteral("a"))), insertAssigns)), - Seq(DeleteAction(Some(_)), UpdateAction(None, secondUpdateAssigns)), + Seq(DeleteAction(Some(_)), UpdateAction(None, secondUpdateAssigns, _)), withSchemaEvolution) => val ti = target.output.find(_.name == "i").get val ts = target.output.find(_.name == "s").get @@ -2282,7 +2282,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { } notMatchedBySourceActions(1) match { case UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("a"))), - Seq(Assignment(us: AttributeReference, IntegerLiteral(1)))) => + Seq(Assignment(us: AttributeReference, IntegerLiteral(1))), _) => // UPDATE condition and assignment are resolved with target table only, so column `s` // and `i` are not ambiguous. val ts = target.output.find(_.name == "s").get @@ -2342,7 +2342,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { AsDataSourceV2Relation(target), AsDataSourceV2Relation(source), EqualTo(IntegerLiteral(1), IntegerLiteral(1)), - Seq(UpdateAction(None, updateAssigns)), // Matched actions + Seq(UpdateAction(None, updateAssigns, _)), // Matched actions Seq(), // Not matched actions Seq(), // Not matched by source actions withSchemaEvolution) => @@ -2395,7 +2395,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { assert(m.matchedActions.length == 1) m.matchedActions.head match { case UpdateAction(_, Seq( - Assignment(_, s1: StaticInvoke), Assignment(_, s2: StaticInvoke))) => + Assignment(_, s1: StaticInvoke), Assignment(_, s2: StaticInvoke)), _) => assert(s1.arguments.length == 2) assert(s1.functionName == "charTypeWriteSideCheck") assert(s2.arguments.length == 2) @@ -2421,7 +2421,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { assert(m.notMatchedBySourceActions.length == 1) m.notMatchedBySourceActions.head match { case UpdateAction(_, Seq( - Assignment(_, s1: StaticInvoke), Assignment(_, s2: StaticInvoke))) => + Assignment(_, s1: StaticInvoke), Assignment(_, s2: StaticInvoke)), _) => assert(s1.arguments.length == 2) assert(s1.functionName == "charTypeWriteSideCheck") assert(s2.arguments.length == 2) From 8c0a10cc9210b8b2829303c8399324ec341c3693 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 22 Nov 2025 09:08:09 -0800 Subject: [PATCH 195/400] [SPARK-54459][SQL][TESTS] Increase `StateStoreSuite.maintenance` test timeout to 2 minutes ### What changes were proposed in this pull request? This PR aims to increase `StateStoreSuite.maintenance` test timeout to 2 minutes for two test suites. - `StateStoreSuite` - `StateStoreSuiteWithRowChecksum` ### Why are the changes needed? To reduce the flakiness. https://github.com/apache/spark/actions/runs/19514807897/job/56065943462 ``` - maintenance *** FAILED *** The code passed to eventually never returned normally. Attempted 833 times over 1.0000272319333332 minutes. Last failure message: StateStore.isLoaded(storeProviderId2) was true. (StateStoreSuite.scala:1032) ``` https://github.com/apache/spark/actions/runs/19588308656/job/56101679879 ``` - maintenance *** FAILED *** The code passed to eventually never returned normally. Attempted 851 times over 1.0017263236166667 minutes. Last failure message: StateStore.isLoaded(storeProviderId2) was true. (StateStoreSuite.scala:1032) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53168 from dongjoon-hyun/SPARK-54459. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 8cfd78d95fa2e83f97831c0d09a345c3506133d0) Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/streaming/state/StateStoreSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 6bb64315e356b..e839ccd35ec0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -922,7 +922,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } } - val timeoutDuration = 1.minute + val timeoutDuration = 2.minutes quietly { withSpark(SparkContext.getOrCreate(conf)) { sc => From cbf2a4238068420ecf4dfa7a925d85bf4595bfe1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 22 Nov 2025 09:09:06 -0800 Subject: [PATCH 196/400] [SPARK-54460][SQL][TESTS] Mark `StateStoreInstanceMetricSuite` as `ExtendedSQLTest` ### What changes were proposed in this pull request? This PR aims to mark `StateStoreInstanceMetricSuite` as `ExtendedSQLTest`. ### Why are the changes needed? To balance the SQL test pipelines by moving the heavy test from `other tests` to `extended tests` pipelines. Screenshot 2025-11-21 at 22 03 00 `StateStoreInstanceMetricSuite` is a good candidate because it took 9 minutes in the GitHub Action environment and local environment. - https://github.com/apache/spark/actions/runs/19588299018/job/56101678778 ``` $ build/sbt "sql/testOnly *.StateStoreInstanceMetricSuite" ... [info] Run completed in 9 minutes, 14 seconds. [info] Total number of tests run: 11 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 11, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [success] Total time: 586 s (0:09:46.0), completed Nov 21, 2025, 10:11:46 PM ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53169 from dongjoon-hyun/SPARK-54460. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 09678f7eba3741ba2e35e1689de6521351ab1623) Signed-off-by: Dongjoon Hyun --- .../streaming/state/StateStoreInstanceMetricSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala index df4d19226b9d3..2db69fda8c8ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.expr import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ +import org.apache.spark.tags.ExtendedSQLTest // RocksDBSkipMaintenanceOnCertainPartitionsProvider is a test-only provider that skips running // maintenance for partitions 0 and 1 (these are arbitrary choices). This is used to test @@ -49,6 +50,7 @@ class HDFSBackedSkipMaintenanceOnCertainPartitionsProvider extends HDFSBackedSta } } +@ExtendedSQLTest class StateStoreInstanceMetricSuite extends StreamTest with AlsoTestWithRocksDBFeatures { import testImplicits._ From 36e056e0f9f0e4c9b2ae4cb41428b8c640d9bff5 Mon Sep 17 00:00:00 2001 From: jameswillis Date: Sat, 22 Nov 2025 15:35:41 -0800 Subject: [PATCH 197/400] [SPARK-53635][SQL] Support Scala UDFs with input args of type Seq[Row] ### What changes were proposed in this pull request? The refactors of encoders in Spark 4 broke support for Scala UDFs that take in an argument of type `Seq[Row]`. This change restores the Spark 3.5 behavior of passing None for the encoder for arguments of that type into `SparkUserDefinedFunction`. ### Why are the changes needed? This change resolves issue SPARK-53635. This will help scala UDF users migrate more smoothly to Spark 4 and maintain backwards compatability for UDFs in Scala. ### Does this PR introduce _any_ user-facing change? Yes. It makes Spark 4 behave like Spark 3 when faced with a Scala UDF with an input argument of type `Seq[Row]` ### How was this patch tested? Existing and new unit tests. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Copilot/GPT-5 Closes #53027 from james-willis/james-willis/SPARK53635. Authored-by: jameswillis Signed-off-by: Dongjoon Hyun (cherry picked from commit ea74e57b260d6549f4e05303f08feb01997ce93a) Signed-off-by: Dongjoon Hyun --- .../sql/expressions/UserDefinedFunction.scala | 15 ++++++++++++--- .../spark/sql/DataFrameComplexTypeSuite.scala | 8 ++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 6a22cbfaf351e..d6da8ce17e832 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.expressions -import scala.reflect.runtime.universe.TypeTag +import scala.reflect.runtime.universe.{typeOf, TypeTag} import scala.util.Try import org.apache.spark.annotation.Stable -import org.apache.spark.sql.{Column, Encoder} +import org.apache.spark.sql.{Column, Encoder, Row} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.internal.{InvokeInlineUserDefinedFunction, UserDefinedFunctionLike} @@ -130,8 +130,17 @@ object SparkUserDefinedFunction { returnTypeTag: TypeTag[_], inputTypeTags: TypeTag[_]*): SparkUserDefinedFunction = { val outputEncoder = ScalaReflection.encoderFor(returnTypeTag) + // Return None for any input that is a Seq[Row]. + // This replicates the Spark 3 behavior of passing None as the encoder for such inputs. + val seqRowType = typeOf[Seq[_]] + val rowType = typeOf[Row] val inputEncoders = inputTypeTags.map { tag => - Try(ScalaReflection.encoderFor(tag)).toOption + val tpe = tag.tpe + if (tpe <:< seqRowType && tpe.typeArgs.nonEmpty && tpe.typeArgs.head =:= rowType) { + None + } else { + Try(ScalaReflection.encoderFor(tag)).toOption + } } SparkUserDefinedFunction( f = function, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 8024b579e5d0c..984ff8fc51dd7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -71,6 +71,14 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession { df.select(map($"a", $"b").as("s")).select(f($"s".getItem("a"))).collect() } + test("SPARK-53635 array of structure to Seq[Row]") { + val f = udf((items: Seq[Row], num: Int) => items.length + num) + val df = spark.sql("SELECT array(struct(1 as a), struct(2 as a), struct(3 as a)) as items") + val result = df.select(f($"items", lit(3)).alias("countNum")).collect() + assert(result.length == 1) + assert(result(0).getAs[Int]("countNum") == 6) + } + test("SPARK-12477 accessing null element in array field") { val df = sparkContext.parallelize(Seq((Seq("val1", null, "val2"), Seq(Some(1), None, Some(2))))).toDF("s", "i") From 9bf0ed144f6f174b2f66deec67e38556e7e97687 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Sat, 22 Nov 2025 15:57:02 -0800 Subject: [PATCH 198/400] [SPARK-54363][REPL][TESTS] Access to buffer in SparkShellSuite should be synchronized between threads ### What changes were proposed in this pull request? Synchronize access to `buffer` in `SparkShellSuite` using existing `lock`. ### Why are the changes needed? `buffer` is updated on `ProcessOutputCapturer` threads and when it is access during exception handling, access to the buffer needs to be locked to avoid using partially updated buffer. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? running test ### Was this patch authored or co-authored using generative AI tooling? No Closes #53074 from vrozov/SPARK-54363. Authored-by: Vlad Rozov Signed-off-by: Dongjoon Hyun (cherry picked from commit f5a55406741304d05629fe5dc8e3d039551e5744) Signed-off-by: Dongjoon Hyun --- .../apache/spark/repl/SparkShellSuite.scala | 38 ++++++++++--------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala index 067f08cb67528..1aad74c8aefc1 100644 --- a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala @@ -95,6 +95,25 @@ class SparkShellSuite extends SparkFunSuite { } } + def handleException(cause: Throwable): Unit = lock.synchronized { + val message = + s""" + |======================= + |SparkShellSuite failure output + |======================= + |Spark Shell command line: ${command.mkString(" ")} + |Exception: $cause + |Failed to capture next expected output "${expectedAnswers(next)}" within $timeout. + | + |${buffer.mkString("\n")} + |=========================== + |End SparkShellSuite failure output + |=========================== + """.stripMargin + logError(message, cause) + fail(message, cause) + } + val process = new ProcessBuilder(command: _*).start() val stdinWriter = new OutputStreamWriter(process.getOutputStream, StandardCharsets.UTF_8) @@ -119,23 +138,8 @@ class SparkShellSuite extends SparkFunSuite { } ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeoutForQuery) log.info("Found all expected output.") - } catch { case cause: Throwable => - val message = - s""" - |======================= - |SparkShellSuite failure output - |======================= - |Spark Shell command line: ${command.mkString(" ")} - |Exception: $cause - |Failed to capture next expected output "${expectedAnswers(next)}" within $timeout. - | - |${buffer.mkString("\n")} - |=========================== - |End SparkShellSuite failure output - |=========================== - """.stripMargin - logError(message, cause) - fail(message, cause) + } catch { + case cause: Throwable => handleException(cause) } finally { if (!process.waitFor(1, MINUTES)) { try { From 97a5c87e839cff851e7c9239812e6717a1935b11 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Sat, 22 Nov 2025 18:20:33 -0800 Subject: [PATCH 199/400] [SPARK-54412][SQL][TESTS] Clean up `v` properly in `identifier-clause.sql` SQL golden file ### What changes were proposed in this pull request? This PR aims to clean up `v` properly in `identifier-clause.sql` SQL golden file. Note that this is originated from #53121 with the original authorship although the code is changed [according to the community review comment](https://github.com/apache/spark/pull/53121#discussion_r2540370700). ### Why are the changes needed? Due to the lack of a proper clean-up, this causes flaky test failures ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53121 Closes #53176 from dongjoon-hyun/SPARK-54412-2. Authored-by: Vlad Rozov Signed-off-by: Dongjoon Hyun (cherry picked from commit 12bce6b7f8f54eea2f2ec88a48787c405cc48706) Signed-off-by: Dongjoon Hyun --- .../analyzer-results/identifier-clause-legacy.sql.out | 6 ++++++ .../sql-tests/analyzer-results/identifier-clause.sql.out | 6 ++++++ .../test/resources/sql-tests/inputs/identifier-clause.sql | 1 + .../sql-tests/results/identifier-clause-legacy.sql.out | 8 ++++++++ .../resources/sql-tests/results/identifier-clause.sql.out | 8 ++++++++ 5 files changed, 29 insertions(+) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out index 2d7d551f51ca5..e53a8153e8292 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -1405,6 +1405,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + -- !query CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 9adecd5e53733..abc6cc625b6aa 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -1379,6 +1379,12 @@ Project [c1#x] +- LocalRelation [col1#x] +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + -- !query CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index 10d14eebc1d47..bb6c7107062d8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -180,6 +180,7 @@ SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v); CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1); SELECT c1 FROM v; +DROP VIEW IF EXISTS v; CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV; INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); SELECT c1 FROM tab; diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out index 79f0f8e77c767..272d9bce81653 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -1557,6 +1557,14 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + -- !query CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 2e2efa2fdc27b..b398c07b14e11 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -1496,6 +1496,14 @@ struct 1 +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + -- !query CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV -- !query schema From 13d62dc60f28c084875ecb87ad59a9d4dd40fbb3 Mon Sep 17 00:00:00 2001 From: AbinayaJayaprakasam Date: Sat, 22 Nov 2025 22:56:45 -0800 Subject: [PATCH 200/400] [SPARK-50072][SQL] Handle ArithmeticException in interval parsing with large values SPARK-50072][SQL] Handle ArithmeticException in interval parsing with large values ### What changes were proposed in this pull request? This PR will fixe SPARK-50072 where parsing intervals with large day values throws an raw java.lang.ArithmeticException instead of a user-friendly SparkArithmeticException. ### Why are the changes needed? When executing queries like 'SELECT interval 106751991 day 4 hour 0 minute 54.776 second', Spark throws a raw ArithmeticException with 'long overflow' message instead of a SparkArithmeticException with error class INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION. This will improve user experience in terms of understanding the error. ### Does this PR introduce any user-facing change? Yes. Users will now see a proper SparkArithmeticException with error class INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION instead of a raw ArithmeticException. ### How was this patch tested? - I Added unit test 'interval overflow with large day values' in IntervalUtilsSuite - Updated existing 'interval duration' test to expect SparkArithmeticException - Added SQL integration test in interval.sql -i have manually tested with spark-shell ### Was this patch authored or co-authored using generative AI tooling? No. ### What changes were proposed in this pull request? ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #53172 from AbinayaJayaprakasam/SPARK-50072-fix. Authored-by: AbinayaJayaprakasam Signed-off-by: Dongjoon Hyun (cherry picked from commit 3861ba498f6d78f9603949dc145395e162f51cdc) Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/util/IntervalUtils.scala | 21 +++++++++------ .../catalyst/util/IntervalUtilsSuite.scala | 27 ++++++++++++++++--- .../analyzer-results/interval.sql.out | 24 ++++++++++++++--- .../analyzer-results/nonansi/interval.sql.out | 24 ++++++++++++++--- .../resources/sql-tests/inputs/interval.sql | 5 ++++ .../sql-tests/results/interval.sql.out | 26 +++++++++++++++--- .../results/nonansi/interval.sql.out | 26 +++++++++++++++--- 7 files changed, 125 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 8793c0407a9b5..ef10a308cff9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -594,14 +594,19 @@ object IntervalUtils extends SparkIntervalUtils { interval: CalendarInterval, targetUnit: TimeUnit, daysPerMonth: Int = 31): Long = { - val monthsDuration = Math.multiplyExact( - daysPerMonth * MICROS_PER_DAY, - interval.months) - val daysDuration = Math.multiplyExact( - MICROS_PER_DAY, - interval.days) - val result = Math.addExact(interval.microseconds, Math.addExact(daysDuration, monthsDuration)) - targetUnit.convert(result, TimeUnit.MICROSECONDS) + try { + val monthsDuration = Math.multiplyExact( + daysPerMonth * MICROS_PER_DAY, + interval.months) + val daysDuration = Math.multiplyExact( + MICROS_PER_DAY, + interval.days) + val result = Math.addExact(interval.microseconds, Math.addExact(daysDuration, monthsDuration)) + targetUnit.convert(result, TimeUnit.MICROSECONDS) + } catch { + case _: ArithmeticException => + throw QueryExecutionErrors.withoutSuggestionIntervalArithmeticOverflowError(context = null) + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index cbaebfa12238a..a87d599711cfc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.util import java.time.{Duration, Period} import java.util.concurrent.TimeUnit -import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} +import org.apache.spark.{SparkArithmeticException, SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros @@ -364,10 +364,29 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { assert(duration("1 microsecond", TimeUnit.MICROSECONDS, 30) === 1) assert(duration("1 month -30 days", TimeUnit.DAYS, 31) === 1) - val e = intercept[ArithmeticException] { - duration(s"${Integer.MAX_VALUE} month", TimeUnit.SECONDS, 31) + checkError( + exception = intercept[SparkArithmeticException] { + duration(s"${Integer.MAX_VALUE} month", TimeUnit.SECONDS, 31) + }, + condition = "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + parameters = Map.empty + ) + } + + test("interval overflow with large day values") { + // Test case for SPARK-50072: handling ArithmeticException during interval parsing + // The value 106751991 days causes overflow when converted to microseconds + def duration(s: String, unit: TimeUnit): Long = { + IntervalUtils.getDuration(stringToInterval(UTF8String.fromString(s)), unit) } - assert(e.getMessage.contains("overflow")) + + checkError( + exception = intercept[SparkArithmeticException] { + duration("106751991 days 4 hours 0 minutes 54.776 seconds", TimeUnit.MICROSECONDS) + }, + condition = "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + parameters = Map.empty + ) } test("negative interval") { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out index c023e3b56f117..88121e4dc3c21 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out @@ -61,15 +61,21 @@ Project [(INTERVAL '2147483647' MONTH / 0.5) AS (INTERVAL '2147483647' MONTH / 0 -- !query select interval 2147483647 day * 2 -- !query analysis -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query select interval 2147483647 day / 0.5 -- !query analysis -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query @@ -3212,3 +3218,13 @@ SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVA -- !query analysis Project [width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, cast(10 as bigint)) AS width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, 10)#xL] +- OneRowRelation + + +-- !query +SELECT interval 106751991 day 4 hour 0 minute 54.776 second +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out index c0196bbe118ef..259cb2bff5ef8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out @@ -61,15 +61,21 @@ Project [(INTERVAL '2147483647' MONTH / 0.5) AS (INTERVAL '2147483647' MONTH / 0 -- !query select interval 2147483647 day * 2 -- !query analysis -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query select interval 2147483647 day / 0.5 -- !query analysis -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query @@ -3212,3 +3218,13 @@ SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVA -- !query analysis Project [width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, cast(10 as bigint)) AS width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, 10)#xL] +- OneRowRelation + + +-- !query +SELECT interval 106751991 day 4 hour 0 minute 54.776 second +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index e4da28c2e7588..e8e10d089dec6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -385,3 +385,8 @@ SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10 SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10); SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10); SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10); + +-- interval overflow with large day values (SPARK-50072) +-- This should throw INTERVAL_ARITHMETIC_OVERFLOW error +SELECT interval 106751991 day 4 hour 0 minute 54.776 second; + diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index c32ecc19448aa..79fce5906df8a 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -75,8 +75,11 @@ select interval 2147483647 day * 2 -- !query schema struct<> -- !query output -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query @@ -84,8 +87,11 @@ select interval 2147483647 day / 0.5 -- !query schema struct<> -- !query output -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query @@ -3897,3 +3903,15 @@ SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVA struct -- !query output 5 + + +-- !query +SELECT interval 106751991 day 4 hour 0 minute 54.776 second +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out index a8a0423bdb3e0..e40053d1bc793 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out @@ -75,8 +75,11 @@ select interval 2147483647 day * 2 -- !query schema struct<> -- !query output -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query @@ -84,8 +87,11 @@ select interval 2147483647 day / 0.5 -- !query schema struct<> -- !query output -java.lang.ArithmeticException -long overflow +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} -- !query @@ -3710,3 +3716,15 @@ SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVA struct -- !query output 5 + + +-- !query +SELECT interval 106751991 day 4 hour 0 minute 54.776 second +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} From c60254e7baba1526c8af0e5cfca46009f182e081 Mon Sep 17 00:00:00 2001 From: Yuheng Chang Date: Sun, 23 Nov 2025 06:04:56 -0800 Subject: [PATCH 201/400] [SPARK-54452] Fix empty response from SparkConnect server for `spark.sql(...)` inside FlowFunction ### What changes were proposed in this pull request? In PR #53024, we added SDP support for `spark.sql(...)` inside a FlowFunction. For these calls, instead of eagerly executing the SQL, the Spark Connect server should return the raw logical plan to the client and defer execution to the flow function. However, in that PR we constructed the response object but forgot to actually return it to the Spark Connect client, so the client received an empty response. This went unnoticed in tests because, when the client sees an empty `spark.sql(...)` response, [it falls back to creating an empty DataFrame holding the raw logical plan](https://github.com/apache/spark/blob/master/python/pyspark/sql/connect/session.py#L829-L835), which happens to match the desired behavior. This PR fixes the bug by returning the proper response instead of relying on that implicit fallback. ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? This PR fixes a bug introduced in #53024 where the server did not return the constructed spark.sql(...) response to the client. ### How was this patch tested? New tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #53156 from SCHJonathan/jonathan-chang_data/fix-spark-sql-bug. Authored-by: Yuheng Chang Signed-off-by: Dongjoon Hyun (cherry picked from commit 997525cae9b00e47626ce09f00e779b279551ace) Signed-off-by: Dongjoon Hyun --- .../connect/planner/SparkConnectPlanner.scala | 8 ++ ...SparkDeclarativePipelinesServerSuite.scala | 94 +++++++++++++++++++ 2 files changed, 102 insertions(+) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 644784fa3db6c..9af2e7cb46616 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -2992,6 +2992,14 @@ class SparkConnectPlanner( // the SQL command and defer the actual analysis and execution to the flow function. if (insidePipelineFlowFunction) { result.setRelation(relation) + executeHolder.eventsManager.postFinished() + responseObserver.onNext( + ExecutePlanResponse + .newBuilder() + .setSessionId(sessionHolder.sessionId) + .setServerSideSessionId(sessionHolder.serverSessionId) + .setSqlCommandResult(result) + .build) return } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala index c9551646385c2..3cb45fa6e1720 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala @@ -850,4 +850,98 @@ class SparkDeclarativePipelinesServerSuite } } } + + test( + "SPARK-54452: spark.sql() inside a pipeline flow function should return a sql_command_result") { + withRawBlockingStub { implicit stub => + val graphId = createDataflowGraph + val pipelineAnalysisContext = proto.PipelineAnalysisContext + .newBuilder() + .setDataflowGraphId(graphId) + .setFlowName("flow1") + .build() + val userContext = proto.UserContext + .newBuilder() + .addExtensions(com.google.protobuf.Any.pack(pipelineAnalysisContext)) + .setUserId("test_user") + .build() + + val relation = proto.Plan + .newBuilder() + .setCommand( + proto.Command + .newBuilder() + .setSqlCommand( + proto.SqlCommand + .newBuilder() + .setInput( + proto.Relation + .newBuilder() + .setRead(proto.Read + .newBuilder() + .setNamedTable( + proto.Read.NamedTable.newBuilder().setUnparsedIdentifier("table")) + .build()) + .build())) + .build()) + .build() + + val sparkSqlRequest = proto.ExecutePlanRequest + .newBuilder() + .setUserContext(userContext) + .setPlan(relation) + .setSessionId(UUID.randomUUID().toString) + .build() + val sparkSqlResponse = stub.executePlan(sparkSqlRequest).next() + assert(sparkSqlResponse.hasSqlCommandResult) + assert( + sparkSqlResponse.getSqlCommandResult.getRelation == + relation.getCommand.getSqlCommand.getInput) + } + } + + test( + "SPARK-54452: spark.sql() outside a pipeline flow function should return a " + + "sql_command_result") { + withRawBlockingStub { implicit stub => + val graphId = createDataflowGraph + val pipelineAnalysisContext = proto.PipelineAnalysisContext + .newBuilder() + .setDataflowGraphId(graphId) + .build() + val userContext = proto.UserContext + .newBuilder() + .addExtensions(com.google.protobuf.Any.pack(pipelineAnalysisContext)) + .setUserId("test_user") + .build() + + val relation = proto.Plan + .newBuilder() + .setCommand( + proto.Command + .newBuilder() + .setSqlCommand( + proto.SqlCommand + .newBuilder() + .setInput(proto.Relation + .newBuilder() + .setSql(proto.SQL.newBuilder().setQuery("SELECT * FROM RANGE(5)")) + .build()) + .build()) + .build()) + .build() + + val sparkSqlRequest = proto.ExecutePlanRequest + .newBuilder() + .setUserContext(userContext) + .setPlan(relation) + .setSessionId(UUID.randomUUID().toString) + .build() + val sparkSqlResponse = stub.executePlan(sparkSqlRequest).next() + assert(sparkSqlResponse.hasSqlCommandResult) + assert( + sparkSqlResponse.getSqlCommandResult.getRelation == + relation.getCommand.getSqlCommand.getInput) + } + } } From 2f8dcb858984263a168c3df632c84c96b3215e17 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 23 Nov 2025 08:02:20 -0800 Subject: [PATCH 202/400] [SPARK-54469][DSTREAM][TESTS] Fix `StreamingContextSuite.stop slow receiver gracefully` test to clean up `SprakContext` ### What changes were proposed in this pull request? This PR aims to fix `StreamingContextSuite.stop slow receiver gracefully` test case to clean up `SprakContext`. ### Why are the changes needed? According to the log, `20000s` setting is propagated to the other test case and causes flaky failures. - https://github.com/apache/spark/actions/runs/19608517338/job/56150848587 ``` - SPARK-22955 graceful shutdown shouldn't lead to job generation error *** FAILED *** The code passed to failAfter did not complete within 20 minutes. (StreamingContextSuite.scala:861) ``` `20000s` setting is originated from `StreamingContextSuite.stop slow receiver gracefully`. We should close the SparkContext correctly. https://github.com/apache/spark/blob/997525cae9b00e47626ce09f00e779b279551ace/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala#L342-L344 ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53182 from dongjoon-hyun/SPARK-54469. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit cda51211f8d475f63865713df26827e9639981ea) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/streaming/StreamingContextSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 564713af88aec..539e74d9a2558 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -365,6 +365,8 @@ class StreamingContextSuite assert(runningCount > 0) assert(runningCount == totalNumRecords) Thread.sleep(100) + sc.stop() + LocalStreamingContext.ensureNoActiveSparkContext() } test ("registering and de-registering of streamingSource") { From 6cb85d9149c36d2903d215106b0c81d4d791e594 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 23 Nov 2025 09:21:03 -0800 Subject: [PATCH 203/400] [SPARK-54470][CORE][TESTS] Fix `BlockManagerDecommissionIntegrationSuite` to wait shuffle migrations ### What changes were proposed in this pull request? This PR aims to fix `BlockManagerDecommissionIntegrationSuite` to wait shuffle migrations to mitigate the flakiness in the slow environment. ### Why are the changes needed? `BlockManagerDecommissionIntegrationSuite` shows a flakiness like the following in a slow environment. - https://github.com/apache/spark/actions/runs/19608517338/job/56150848584 ``` - SPARK-46957: Migrated shuffle files should be able to cleanup from executor *** FAILED *** 0 was not greater than or equal to 12 (BlockManagerDecommissionIntegrationSuite.scala:422) ``` It's because line 412 only ensures the start of migration. We need to wait more to proceed the next step, line 422. https://github.com/apache/spark/blob/997525cae9b00e47626ce09f00e779b279551ace/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala#L409-L422 ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53183 from dongjoon-hyun/SPARK-54470. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 5f1d62472f035d50300caf5f7883a5d61c18e9b6) Signed-off-by: Dongjoon Hyun --- .../storage/BlockManagerDecommissionIntegrationSuite.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 2b2a67c3c00ad..8c6b9cc288ec1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -418,8 +418,11 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS .map(_.asInstanceOf[ShuffleIndexBlockId].shuffleId) .get + eventually(timeout(1.minute), interval(10.milliseconds)) { + val newShuffleFiles = shuffleFiles.diff(existingShuffleFiles) + assert(newShuffleFiles.size >= shuffleBlockUpdates.size) + } val newShuffleFiles = shuffleFiles.diff(existingShuffleFiles) - assert(newShuffleFiles.size >= shuffleBlockUpdates.size) // Remove the shuffle data sc.shuffleDriverComponents.removeShuffle(shuffleId, true) From 8c06c6c8bff935421e2a387c14511198b14366f8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 23 Nov 2025 09:22:47 -0800 Subject: [PATCH 204/400] [SPARK-54471][INFRA] Remove `build_python_3.11_macos.yml` CI ### What changes were proposed in this pull request? This PR aims to remove `build_python_3.11_macos.yml` (MacOS 15) CI in favor of `build_python_3.11_macos26.yml` (MacOS 25). ### Why are the changes needed? Since we successfully migrated to `MacOS25` from `MacOS15`, we can remove the old OS to meet SPARK-48094 (Reduce GitHub Action usage according to ASF project allowance). - https://github.com/apache/spark/actions/workflows/build_python_3.11_macos26.yml ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review because this is a removal of CI. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53184 from dongjoon-hyun/SPARK-54471. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 021f89ae3d13ca9b48acc12188d4746065674718) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_python_3.11_macos.yml | 33 ------------------- README.md | 1 - 2 files changed, 34 deletions(-) delete mode 100644 .github/workflows/build_python_3.11_macos.yml diff --git a/.github/workflows/build_python_3.11_macos.yml b/.github/workflows/build_python_3.11_macos.yml deleted file mode 100644 index 9566bfd8271d1..0000000000000 --- a/.github/workflows/build_python_3.11_macos.yml +++ /dev/null @@ -1,33 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -name: "Build / Python-only (master, Python 3.11, MacOS)" - -on: - schedule: - - cron: '0 21 * * *' - workflow_dispatch: - -jobs: - run-build: - permissions: - packages: write - name: Run - uses: ./.github/workflows/python_hosted_runner_test.yml - if: github.repository == 'apache/spark' diff --git a/README.md b/README.md index f3435178c68eb..77d12031106dd 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,6 @@ This README file only contains basic setup instructions. | | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_3.10.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_3.10.yml) | | | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_3.11_classic_only.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_3.11_classic_only.yml) | | | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_3.11_arm.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_3.11_arm.yml) | -| | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_3.11_macos.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_3.11_macos.yml) | | | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_3.11_macos26.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_3.11_macos26.yml) | | | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_numpy_2.1.3.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_numpy_2.1.3.yml) | | | [![GitHub Actions Build](https://github.com/apache/spark/actions/workflows/build_python_3.12.yml/badge.svg)](https://github.com/apache/spark/actions/workflows/build_python_3.12.yml) | From 0366edbf513961ea5a406d59d8591facb52b2a4e Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sun, 23 Nov 2025 14:21:55 -0800 Subject: [PATCH 205/400] [SPARK-54022][SQL] Make DSv2 table resolution aware of cached tables ### What changes were proposed in this pull request? This PR makes DSv2 table resolution aware of cached tables via CACHE TABLE t or `spark.table("t").cache()` commands. ### Why are the changes needed? These changes are needed to avoid silent cache misses for DSv2 tables. Cache lookups depend on DSv2 Table instance equality. If each query is allowed to load a new Table instance from the metastore, this would mean connectors can pick up external changes, leading to unexpected cache misses. This contradicts the behavior we had for built-in Tables and some DSv1 connectors such as Delta. Historically, the expected behavior of CACHE TABLE t and `spark.table("t").cache()` is to cache the table state. ### Does this PR introduce _any_ user-facing change? Yes. The PR fixes the the resolution for DSv2 so that CACHE TABLE t behaves correctly and reliably. - caching table via Dataset API will now pin table state - caching table via CACHE TABLE will now pin table state - caching a query via Dataset API will continue to simply cache the query plan as before ### How was this patch tested? This PR comes with tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52764 from aokolnychyi/spark-54022. Lead-authored-by: Anton Okolnychyi Co-authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun (cherry picked from commit 92c948f4137686e3a566a58f6e671bc0c4a9cce5) Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 10 ++- .../sql/catalyst/analysis/RelationCache.scala | 28 +++++++ .../analysis/RelationResolution.scala | 74 ++++++++++++------ .../analysis/resolver/HybridAnalyzer.scala | 1 + .../catalyst/analysis/resolver/Resolver.scala | 13 +++- .../catalyst/plans/logical/v2Commands.scala | 8 +- .../sql/connector/catalog/CatalogV2Util.scala | 24 +++++- .../spark/sql/execution/CacheManager.scala | 75 ++++++++++++++++--- .../spark/sql/execution/QueryExecution.scala | 2 +- .../datasources/v2/V2TableRefreshUtil.scala | 30 ++++++-- .../v2/WriteToDataSourceV2Exec.scala | 9 ++- .../internal/BaseSessionStateBuilder.scala | 4 +- .../spark/sql/internal/SharedState.scala | 8 ++ .../apache/spark/sql/CachedTableSuite.scala | 35 +++++++++ .../sql/analysis/resolver/ResolverSuite.scala | 5 +- .../DataSourceV2DataFrameSuite.scala | 67 +++++++++++++++++ .../command/PlanResolutionSuite.scala | 4 +- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 18 files changed, 337 insertions(+), 62 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationCache.scala rename sql/{catalyst => core}/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala (79%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fb8a84a85fc89..3b916a62b3886 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -80,7 +80,8 @@ object SimpleAnalyzer extends Analyzer( FunctionRegistry.builtin, TableFunctionRegistry.builtin) { override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = {} - })) { + }), + RelationCache.empty) { override def resolver: Resolver = caseSensitiveResolution } @@ -285,11 +286,14 @@ object Analyzer { * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and * [[UnresolvedRelation]]s into fully typed objects using information in a [[SessionCatalog]]. */ -class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor[LogicalPlan] +class Analyzer( + override val catalogManager: CatalogManager, + private[sql] val sharedRelationCache: RelationCache = RelationCache.empty) + extends RuleExecutor[LogicalPlan] with CheckAnalysis with AliasHelper with SQLConfHelper with ColumnResolutionHelper { private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog - private val relationResolution = new RelationResolution(catalogManager) + private val relationResolution = new RelationResolution(catalogManager, sharedRelationCache) private val functionResolution = new FunctionResolution(catalogManager, relationResolution) override protected def validatePlanChanges( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationCache.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationCache.scala new file mode 100644 index 0000000000000..770a5e780b24a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationCache.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +private[sql] trait RelationCache { + def lookup(nameParts: Seq[String], resolver: Resolver): Option[LogicalPlan] +} + +private[sql] object RelationCache { + val empty: RelationCache = (_, _) => None +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala index c7b92bc2a9fe5..15d5e4874dbb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala @@ -46,7 +46,9 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ -class RelationResolution(override val catalogManager: CatalogManager) +class RelationResolution( + override val catalogManager: CatalogManager, + sharedRelationCache: RelationCache) extends DataTypeErrorsBase with Logging with LookupCatalog @@ -118,36 +120,62 @@ class RelationResolution(override val catalogManager: CatalogManager) val planId = u.getTagValue(LogicalPlan.PLAN_ID_TAG) relationCache .get(key) - .map { cache => - val cachedRelation = cache.transform { - case multi: MultiInstanceRelation => - val newRelation = multi.newInstance() - newRelation.copyTagsFrom(multi) - newRelation - } - cloneWithPlanId(cachedRelation, planId) - } + .map(adaptCachedRelation(_, planId)) .orElse { - val writePrivilegesString = - Option(u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES)) - val table = - CatalogV2Util.loadTable(catalog, ident, finalTimeTravelSpec, writePrivilegesString) - val loaded = createRelation( + val writePrivileges = u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES) + val finalOptions = u.clearWritePrivileges.options + val table = CatalogV2Util.loadTable( catalog, ident, - table, - u.clearWritePrivileges.options, - u.isStreaming, - finalTimeTravelSpec - ) - loaded.foreach(relationCache.update(key, _)) - loaded.map(cloneWithPlanId(_, planId)) - } + finalTimeTravelSpec, + Option(writePrivileges)) + + val sharedRelationCacheMatch = for { + t <- table + if finalTimeTravelSpec.isEmpty && writePrivileges == null && !u.isStreaming + cached <- lookupSharedRelationCache(catalog, ident, t) + } yield { + val updatedRelation = cached.copy(options = finalOptions) + val nameParts = ident.toQualifiedNameParts(catalog) + val aliasedRelation = SubqueryAlias(nameParts, updatedRelation) + relationCache.update(key, aliasedRelation) + adaptCachedRelation(aliasedRelation, planId) + } + + sharedRelationCacheMatch.orElse { + val loaded = createRelation( + catalog, + ident, + table, + finalOptions, + u.isStreaming, + finalTimeTravelSpec) + loaded.foreach(relationCache.update(key, _)) + loaded.map(cloneWithPlanId(_, planId)) + } + } case _ => None } } } + private def lookupSharedRelationCache( + catalog: CatalogPlugin, + ident: Identifier, + table: Table): Option[DataSourceV2Relation] = { + CatalogV2Util.lookupCachedRelation(sharedRelationCache, catalog, ident, table, conf) + } + + private def adaptCachedRelation(cached: LogicalPlan, planId: Option[Long]): LogicalPlan = { + val plan = cached transform { + case multi: MultiInstanceRelation => + val newRelation = multi.newInstance() + newRelation.copyTagsFrom(multi) + newRelation + } + cloneWithPlanId(plan, planId) + } + private def createRelation( catalog: CatalogPlugin, ident: Identifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala index 0117b3fc2fb55..d346969be8eff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala @@ -302,6 +302,7 @@ object HybridAnalyzer { resolverGuard = new ResolverGuard(legacyAnalyzer.catalogManager), resolver = new Resolver( catalogManager = legacyAnalyzer.catalogManager, + sharedRelationCache = legacyAnalyzer.sharedRelationCache, extensions = legacyAnalyzer.singlePassResolverExtensions, metadataResolverExtensions = legacyAnalyzer.singlePassMetadataResolverExtensions, externalRelationResolution = Some(legacyAnalyzer.getRelationResolution) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala index 75d23f29ecfc5..78029d593df13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.analysis.{ AnalysisErrorAt, FunctionResolution, MultiInstanceRelation, + RelationCache, RelationResolution, ResolvedInlineTable, UnresolvedHaving, @@ -71,6 +72,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors */ class Resolver( catalogManager: CatalogManager, + sharedRelationCache: RelationCache = RelationCache.empty, override val extensions: Seq[ResolverExtension] = Seq.empty, metadataResolverExtensions: Seq[ResolverExtension] = Seq.empty, externalRelationResolution: Option[RelationResolution] = None) @@ -81,8 +83,9 @@ class Resolver( private val cteRegistry = new CteRegistry private val subqueryRegistry = new SubqueryRegistry private val identifierAndCteSubstitutor = new IdentifierAndCteSubstitutor - private val relationResolution = - externalRelationResolution.getOrElse(Resolver.createRelationResolution(catalogManager)) + private val relationResolution = externalRelationResolution.getOrElse { + Resolver.createRelationResolution(catalogManager, sharedRelationCache) + } private val functionResolution = new FunctionResolution(catalogManager, relationResolution) private val expressionResolver = new ExpressionResolver(this, functionResolution, planLogger) private val aggregateResolver = new AggregateResolver(this, expressionResolver) @@ -788,7 +791,9 @@ object Resolver { /** * Create a new instance of the [[RelationResolution]]. */ - def createRelationResolution(catalogManager: CatalogManager): RelationResolution = { - new RelationResolution(catalogManager) + def createRelationResolution( + catalogManager: CatalogManager, + sharedRelationCache: RelationCache = RelationCache.empty): RelationResolution = { + new RelationResolution(catalogManager, sharedRelationCache) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 26ce138523e7e..3f9e8da21d287 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -40,7 +40,6 @@ import org.apache.spark.sql.connector.write.RowLevelOperation.Command.{DELETE, M import org.apache.spark.sql.errors.DataTypeErrors.toSQLType import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, ExtractV2Table} -import org.apache.spark.sql.execution.datasources.v2.V2TableRefreshUtil import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, AtomicType, BooleanType, DataType, IntegerType, MapType, MetadataBuilder, StringType, StructField, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -689,12 +688,7 @@ case class ReplaceTableAsSelect( extends V2CreateTableAsSelectPlan { override def markAsAnalyzed(ac: AnalysisContext): LogicalPlan = { - // RTAS may drop and recreate table before query execution, breaking self-references - // refresh and pin versions here to read from original table versions instead of - // newly created empty table that is meant to serve as target for append/overwrite - val refreshedQuery = V2TableRefreshUtil.refresh(query, versionedOnly = true) - val pinnedQuery = V2TableRefreshUtil.pinVersions(refreshedQuery) - copy(query = pinnedQuery, isAnalyzed = true) + copy(isAnalyzed = true) } override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 28bca400f5b8b..07cb370d18dd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkException, SparkIllegalArgumentException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.CurrentUserContext -import org.apache.spark.sql.catalyst.analysis.{AsOfTimestamp, AsOfVersion, NamedRelation, NoSuchDatabaseException, NoSuchFunctionException, NoSuchTableException, TimeTravelSpec} +import org.apache.spark.sql.catalyst.analysis.{AsOfTimestamp, AsOfVersion, NamedRelation, NoSuchDatabaseException, NoSuchFunctionException, NoSuchTableException, RelationCache, TimeTravelSpec} import org.apache.spark.sql.catalyst.catalog.ClusterBySpec import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, V2ExpressionUtils} import org.apache.spark.sql.catalyst.plans.logical.{SerdeInfo, TableSpec} @@ -36,6 +36,7 @@ import org.apache.spark.sql.connector.catalog.constraints.Constraint import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.expressions.{ClusterByTransform, LiteralValue, Transform} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, MapType, Metadata, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ @@ -497,6 +498,27 @@ private[sql] object CatalogV2Util { loadTable(catalog, ident).map(DataSourceV2Relation.create(_, Some(catalog), Some(ident))) } + def isSameTable( + rel: DataSourceV2Relation, + catalog: CatalogPlugin, + ident: Identifier, + table: Table): Boolean = { + rel.catalog.contains(catalog) && rel.identifier.contains(ident) && rel.table.id == table.id + } + + def lookupCachedRelation( + cache: RelationCache, + catalog: CatalogPlugin, + ident: Identifier, + table: Table, + conf: SQLConf): Option[DataSourceV2Relation] = { + val nameParts = ident.toQualifiedNameParts(catalog) + val cached = cache.lookup(nameParts, conf.resolver) + cached.collect { + case r: DataSourceV2Relation if isSameTable(r, catalog, ident, table) => r + } + } + def isSessionCatalog(catalog: CatalogPlugin): Boolean = { catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index a35efd96060f9..5a38751b61e12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.analysis.V2TableReference import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} @@ -30,13 +31,14 @@ import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPla import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.classic.{Dataset, SparkSession} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper +import org.apache.spark.sql.connector.catalog.CatalogPlugin +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.{IdentifierHelper, MultipartIdentifierHelper} +import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation, LogicalRelationWithTable} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, ExtractV2Table, FileTable} -import org.apache.spark.sql.execution.datasources.v2.V2TableRefreshUtil +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, ExtractV2CatalogAndIdentifier, ExtractV2Table, FileTable, V2TableRefreshUtil} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -240,31 +242,51 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { name: Seq[String], conf: SQLConf, includeTimeTravel: Boolean): Boolean = { - def isSameName(nameInCache: Seq[String]): Boolean = { - nameInCache.length == name.length && nameInCache.zip(name).forall(conf.resolver.tupled) - } + isMatchedTableOrView(plan, name, conf.resolver, includeTimeTravel) + } + + private def isMatchedTableOrView( + plan: LogicalPlan, + name: Seq[String], + resolver: Resolver, + includeTimeTravel: Boolean): Boolean = { EliminateSubqueryAliases(plan) match { case LogicalRelationWithTable(_, Some(catalogTable)) => - isSameName(catalogTable.identifier.nameParts) + isSameName(name, catalogTable.identifier.nameParts, resolver) case DataSourceV2Relation(_, _, Some(catalog), Some(v2Ident), _, timeTravelSpec) => val nameInCache = v2Ident.toQualifiedNameParts(catalog) - isSameName(nameInCache) && (includeTimeTravel || timeTravelSpec.isEmpty) + isSameName(name, nameInCache, resolver) && (includeTimeTravel || timeTravelSpec.isEmpty) case r: V2TableReference => - isSameName(r.identifier.toQualifiedNameParts(r.catalog)) + isSameName(name, r.identifier.toQualifiedNameParts(r.catalog), resolver) case v: View => - isSameName(v.desc.identifier.nameParts) + isSameName(name, v.desc.identifier.nameParts, resolver) case HiveTableRelation(catalogTable, _, _, _, _) => - isSameName(catalogTable.identifier.nameParts) + isSameName(name, catalogTable.identifier.nameParts, resolver) case _ => false } } + private def isSameName( + name: Seq[String], + catalog: CatalogPlugin, + ident: Identifier, + resolver: Resolver): Boolean = { + isSameName(name, ident.toQualifiedNameParts(catalog), resolver) + } + + private def isSameName( + name: Seq[String], + nameInCache: Seq[String], + resolver: Resolver): Boolean = { + nameInCache.length == name.length && nameInCache.zip(name).forall(resolver.tupled) + } + private def uncacheByCondition( spark: SparkSession, isMatchedPlan: LogicalPlan => Boolean, @@ -354,7 +376,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { cd.cachedRepresentation.cacheBuilder.clearCache() val sessionWithConfigsOff = getOrCloneSessionWithConfigsOff(spark) val (newKey, newCache) = sessionWithConfigsOff.withActive { - val refreshedPlan = V2TableRefreshUtil.refresh(cd.plan) + val refreshedPlan = V2TableRefreshUtil.refresh(sessionWithConfigsOff, cd.plan) val qe = sessionWithConfigsOff.sessionState.executePlan(refreshedPlan) qe.normalized -> InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) } @@ -371,6 +393,35 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { } } + private[sql] def lookupCachedTable( + name: Seq[String], + resolver: Resolver): Option[LogicalPlan] = { + val cachedRelations = findCachedRelations(name, resolver) + cachedRelations match { + case cachedRelation +: _ => + CacheManager.logCacheOperation( + log"Relation cache hit for table ${MDC(TABLE_NAME, name.quoted)}") + Some(cachedRelation) + case _ => + None + } + } + + private def findCachedRelations( + name: Seq[String], + resolver: Resolver): Seq[LogicalPlan] = { + cachedData.flatMap { cd => + val plan = EliminateSubqueryAliases(cd.plan) + plan match { + case r @ ExtractV2CatalogAndIdentifier(catalog, ident) + if isSameName(name, catalog, ident, resolver) && r.timeTravelSpec.isEmpty => + Some(r) + case _ => + None + } + } + } + /** * Optionally returns cached data for the given [[Dataset]] */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 12fce2f91dac8..26d2078791aad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -207,7 +207,7 @@ class QueryExecution( // there may be delay between analysis and subsequent phases // therefore, refresh captured table versions to reflect latest data private val lazyTableVersionsRefreshed = LazyTry { - V2TableRefreshUtil.refresh(commandExecuted, versionedOnly = true) + V2TableRefreshUtil.refresh(sparkSession, commandExecuted, versionedOnly = true) } private[sql] def tableVersionsRefreshed: LogicalPlan = lazyTableVersionsRefreshed.get diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala similarity index 79% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala index 0852043cb8228..7071e6fd4ff51 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala @@ -23,7 +23,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.AsOfVersion import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog, V2TableUtil} +import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.errors.QueryCompilationErrors private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { @@ -59,19 +61,29 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { * Tables with time travel specifications are skipped as they reference a specific point * in time and don't have to be refreshed. * + * @param spark the currently active Spark session * @param plan the logical plan to refresh * @param versionedOnly indicates whether to refresh only versioned tables * @return plan with refreshed table metadata */ - def refresh(plan: LogicalPlan, versionedOnly: Boolean = false): LogicalPlan = { - val cache = mutable.HashMap.empty[(TableCatalog, Identifier), Table] + def refresh( + spark: SparkSession, + plan: LogicalPlan, + versionedOnly: Boolean = false): LogicalPlan = { + val currentTables = mutable.HashMap.empty[(TableCatalog, Identifier), Table] plan transform { case r @ ExtractV2CatalogAndIdentifier(catalog, ident) if (r.isVersioned || !versionedOnly) && r.timeTravelSpec.isEmpty => - val currentTable = cache.getOrElseUpdate((catalog, ident), { + val currentTable = currentTables.getOrElseUpdate((catalog, ident), { val tableName = V2TableUtil.toQualifiedName(catalog, ident) - logDebug(s"Refreshing table metadata for $tableName") - catalog.loadTable(ident) + lookupCachedRelation(spark, catalog, ident, r.table) match { + case Some(cached) => + logDebug(s"Refreshing table metadata for $tableName using shared relation cache") + cached.table + case None => + logDebug(s"Refreshing table metadata for $tableName using catalog") + catalog.loadTable(ident) + } }) validateTableIdentity(currentTable, r) validateDataColumns(currentTable, r) @@ -80,6 +92,14 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { } } + private def lookupCachedRelation( + spark: SparkSession, + catalog: TableCatalog, + ident: Identifier, + table: Table): Option[DataSourceV2Relation] = { + CatalogV2Util.lookupCachedRelation(spark.sharedState.relationCache, catalog, ident, table, conf) + } + private def validateTableIdentity(currentTable: Table, relation: DataSourceV2Relation): Unit = { if (relation.table.id != null && relation.table.id != currentTable.id) { throw QueryCompilationErrors.tableIdChangedAfterAnalysis( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 1d7566ce7f3ef..9e5e45e984eb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -168,6 +168,11 @@ case class ReplaceTableAsSelectExec( // 1. Creating the new table fails, // 2. Writing to the new table fails, // 3. The table returned by catalog.createTable doesn't support writing. + // + // RTAS must refresh and pin versions in query to read from original table versions instead of + // newly created empty table that is meant to serve as target for append/overwrite + val refreshedQuery = V2TableRefreshUtil.refresh(session, query, versionedOnly = true) + val pinnedQuery = V2TableRefreshUtil.pinVersions(refreshedQuery) if (catalog.tableExists(ident)) { invalidateCache(catalog, ident) catalog.dropTable(ident) @@ -175,13 +180,13 @@ case class ReplaceTableAsSelectExec( throw QueryCompilationErrors.cannotReplaceMissingTableError(ident) } val tableInfo = new TableInfo.Builder() - .withColumns(getV2Columns(query.schema, catalog.useNullableQuerySchema)) + .withColumns(getV2Columns(pinnedQuery.schema, catalog.useNullableQuerySchema)) .withPartitions(partitioning.toArray) .withProperties(properties.asJava) .build() val table = Option(catalog.createTable(ident, tableInfo)) .getOrElse(catalog.loadTable(ident, Set(TableWritePrivilege.INSERT).asJava)) - writeToTable(catalog, table, writeOptions, ident, query, overwrite = true) + writeToTable(catalog, table, writeOptions, ident, pinnedQuery, overwrite = true) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index c967497b660c7..ef829eaae68c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -176,6 +176,8 @@ abstract class BaseSessionStateBuilder( protected lazy val catalogManager = new CatalogManager(v2SessionCatalog, catalog) + protected lazy val sharedRelationCache = session.sharedState.relationCache + /** * Interface exposed to the user for registering user-defined functions. * @@ -197,7 +199,7 @@ abstract class BaseSessionStateBuilder( * * Note: this depends on the `conf` and `catalog` fields. */ - protected def analyzer: Analyzer = new Analyzer(catalogManager) { + protected def analyzer: Analyzer = new Analyzer(catalogManager, sharedRelationCache) { override val hintResolutionRules: Seq[Rule[LogicalPlan]] = customHintResolutionRules diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index af1f38caab686..8e641294bf8cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.{FsUrlStreamHandlerFactory, Path} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, PATH, VALUE} +import org.apache.spark.sql.catalyst.analysis.RelationCache import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.CacheManager @@ -96,6 +97,13 @@ private[sql] class SharedState( */ val cacheManager: CacheManager = new CacheManager + /** + * A relation cache backed by the cache manager. + */ + private[sql] val relationCache: RelationCache = { + (nameParts, resolver) => cacheManager.lookupCachedTable(nameParts, resolver) + } + /** A global lock for all streaming query lifecycle tracking and management. */ private[sql] val activeQueriesLock = new Object diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 7faf580b6f7f1..12d26c4e195f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.io.{File, FilenameFilter} import java.nio.file.{Files, Paths} import java.time.{Duration, LocalDateTime, LocalTime, Period} +import java.util import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable.HashSet @@ -41,6 +42,8 @@ import org.apache.spark.sql.connector.catalog.CatalogPlugin import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.InMemoryCatalog +import org.apache.spark.sql.connector.catalog.TableWritePrivilege +import org.apache.spark.sql.connector.catalog.TruncatableTable import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, RDDScanExec, SparkPlan, SparkPlanInfo} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, AQEPropagateEmptyRelation} import org.apache.spark.sql.execution.columnar._ @@ -2563,6 +2566,38 @@ class CachedTableSuite extends QueryTest with SQLTestUtils context = ExpectedContext("non_existent", 14, 25)) } + test("SPARK-54022: caching table via CACHE TABLE should pin table state") { + val t = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + withTable(t) { + sql(s"CREATE TABLE $t (id INT, value INT, category STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 10, 'A'), (2, 20, 'B'), (3, 30, 'A')") + + // cache table + sql(s"CACHE TABLE $t") + + // verify caching works as expected + assertCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(1, 10, "A"), Row(2, 20, "B"), Row(3, 30, "A"))) + + // modify table directly to mimic external changes + val tableCatalog = catalog("testcat").asTableCatalog + val table = tableCatalog.loadTable(ident, util.Set.of(TableWritePrivilege.DELETE)) + table.asInstanceOf[TruncatableTable].truncateTable() + + // verify this has no impact on cached state + assertCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(1, 10, "A"), Row(2, 20, "B"), Row(3, 30, "A"))) + + // add more data within session that should invalidate cache + sql(s"INSERT INTO $t VALUES (10, 100, 'x')") + + // table should be re-cached correctly + assertCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(10, 100, "x"))) + } + } + private def cacheManager = spark.sharedState.cacheManager private def pinTable( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala index 8a54f65209748..0e23f984d922b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala @@ -108,7 +108,10 @@ class ResolverSuite extends QueryTest with SharedSparkSession { } private def createResolver(extensions: Seq[ResolverExtension] = Seq.empty): Resolver = { - new Resolver(spark.sessionState.catalogManager, extensions) + new Resolver( + spark.sessionState.catalogManager, + spark.sharedState.relationCache, + extensions) } private class TestRelationResolver extends ResolverExtension { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index d41af4f1465e5..e4b45ec66462a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector +import java.util import java.util.Collections import scala.jdk.CollectionConverters._ @@ -29,6 +30,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSel import org.apache.spark.sql.connector.catalog.{Column, ColumnDefaultValue, DefaultValue, Identifier, InMemoryTableCatalog, TableInfo} import org.apache.spark.sql.connector.catalog.BasicInMemoryTableCatalog import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, UpdateColumnDefaultValue} +import org.apache.spark.sql.connector.catalog.TableWritePrivilege +import org.apache.spark.sql.connector.catalog.TruncatableTable import org.apache.spark.sql.connector.expressions.{ApplyTransform, GeneralScalarExpression, LiteralValue, Transform} import org.apache.spark.sql.connector.expressions.filter.{AlwaysFalse, AlwaysTrue} import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} @@ -1640,6 +1643,70 @@ class DataSourceV2DataFrameSuite } } + test("SPARK-54022: caching table via Dataset API should pin table state") { + val t = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + withTable(t) { + sql(s"CREATE TABLE $t (id INT, value INT, category STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 10, 'A'), (2, 20, 'B'), (3, 30, 'A')") + + // cache table + spark.table(t).cache() + + // verify caching works as expected + assertCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(1, 10, "A"), Row(2, 20, "B"), Row(3, 30, "A"))) + + // modify table directly to mimic external changes + val table = catalog("testcat").loadTable(ident, util.Set.of(TableWritePrivilege.DELETE)) + table.asInstanceOf[TruncatableTable].truncateTable() + + // verify external changes have no impact on cached state + assertCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(1, 10, "A"), Row(2, 20, "B"), Row(3, 30, "A"))) + + // add more data within session that should invalidate cache + sql(s"INSERT INTO $t VALUES (10, 100, 'x')") + + // table should be re-cached correctly + assertCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(10, 100, "x"))) + } + } + + test("SPARK-54022: caching a query via Dataset API should not pin table state") { + val t = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + withTable(t) { + sql(s"CREATE TABLE $t (id INT, value INT, category STRING) USING foo") + sql(s"INSERT INTO $t VALUES (1, 10, 'A'), (2, 20, 'B'), (3, 30, 'A')") + + // cache query on top of table + val df = spark.table(t).select("id") + df.cache() + + // verify query caching works as expected + assertCached(spark.table(t).select("id")) + checkAnswer(spark.table(t).select("id"), Seq(Row(1), Row(2), Row(3))) + + // verify table itself is not cached + assertNotCached(spark.table(t)) + checkAnswer(spark.table(t), Seq(Row(1, 10, "A"), Row(2, 20, "B"), Row(3, 30, "A"))) + + // modify table directly to mimic external changes + val table = catalog("testcat").loadTable(ident, util.Set.of(TableWritePrivilege.DELETE)) + table.asInstanceOf[TruncatableTable].truncateTable() + + // verify cached DataFrame is unaffected by external changes + assertCached(df) + checkAnswer(df, Seq(Row(1), Row(2), Row(3))) + + // verify external changes are reflected correctly when table is queried + assertNotCached(spark.table(t)) + checkAnswer(spark.table(t), Seq.empty) + } + } + private def pinTable(catalogName: String, ident: Identifier, version: String): Unit = { catalog(catalogName) match { case inMemory: BasicInMemoryTableCatalog => inMemory.pinTable(ident, version) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index fb56cecb05ae4..18042bf73adf2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -3284,7 +3284,9 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { unresolvedRelation: UnresolvedRelation, timeTravelSpec: Option[TimeTravelSpec] = None, planId: Option[Long] = None): DataSourceV2Relation = { - val rule = new RelationResolution(catalogManagerWithDefault) + val rule = new RelationResolution( + catalogManagerWithDefault, + spark.sharedState.relationCache) rule.resolveRelation(unresolvedRelation, timeTravelSpec) match { case Some(p @ AsDataSourceV2Relation(relation)) => assert(unresolvedRelation.getTagValue(LogicalPlan.PLAN_ID_TAG) == planId) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index aa801b6e2f680..dec947651dd6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -85,7 +85,7 @@ class HiveSessionStateBuilder( /** * A logical query plan `Analyzer` with rules specific to Hive. */ - override protected def analyzer: Analyzer = new Analyzer(catalogManager) { + override protected def analyzer: Analyzer = new Analyzer(catalogManager, sharedRelationCache) { override val singlePassResolverExtensions: Seq[ResolverExtension] = Seq( new LogicalRelationResolver, new HiveTableRelationNoopResolver From 8b6a2ff9ca54f1639790e178225af9b534db7348 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Mon, 24 Nov 2025 01:09:23 +0000 Subject: [PATCH 206/400] Removing test jars and class files --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 3827 -> 0 bytes core/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes data/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes data/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes dev/test-classes.txt | 7 ------- dev/test-jars.txt | 17 ----------------- .../src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 5674 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../artifact-tests/smallClassFileDup.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 6119 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../artifact-tests/HelloWithPackage.class | Bin 635 -> 0 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 1333 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 5545 -> 0 bytes .../src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 35660 -> 0 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 7439 -> 0 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../test/resources/data/files/TestSerDe.jar | Bin 3838 -> 0 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 19579 -> 0 bytes 26 files changed, 24 deletions(-) delete mode 100644 core/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 core/src/test/resources/TestHelloV3_2.13.jar delete mode 100644 core/src/test/resources/TestUDTF.jar delete mode 100755 data/artifact-tests/junitLargeJar.jar delete mode 100755 data/artifact-tests/smallJar.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar delete mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar delete mode 100644 sql/core/src/test/resources/SPARK-33084.jar delete mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class delete mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class delete mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class delete mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class delete mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar delete mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar delete mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar delete mode 100644 sql/hive/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar delete mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar deleted file mode 100644 index 0c292e7d81ad741dbc11177224b9c30519b7a362..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar deleted file mode 100644 index 514f2d5d26fd358ad5647e0e75edb8ce77b69e30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar deleted file mode 100755 index 3c4930e8e9549d17b89cd3ecb028f048f331797c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar deleted file mode 100644 index 61e1663ad3a2804eb7c39960ea51e56a0ebfaf94..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class deleted file mode 100644 index f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class deleted file mode 100644 index 75a41446cfca1f93570ab90a74d80b51e0fb59e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar deleted file mode 100644 index 768b2334db5c3aa8b1e4186af5fde86120d3231b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..3c4930e8e9549d17b89cd3ecb028f048f331797c GIT binary patch literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar new file mode 100644 index 0000000000000000000000000000000000000000..61e1663ad3a2804eb7c39960ea51e56a0ebfaf94 GIT binary patch literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class new file mode 100644 index 0000000000000000000000000000000000000000..f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce GIT binary patch literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class new file mode 100644 index 0000000000000000000000000000000000000000..75a41446cfca1f93570ab90a74d80b51e0fb59e4 GIT binary patch literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar new file mode 100644 index 0000000000000000000000000000000000000000..768b2334db5c3aa8b1e4186af5fde86120d3231b GIT binary patch literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

+ * When variant columns are accessed with specific field extractions (e.g., variant_get, + * try_variant_get), the optimizer can push these extractions down to the data source. + * The data source can then read only the required fields from variant columns, reducing + * I/O and improving performance. + *

+ * Each {@link VariantExtraction} in the input array represents one field extraction operation. + * Data sources should examine each extraction and determine which ones can be handled efficiently. + * The return value is a boolean array of the same length, where each element indicates whether + * the corresponding extraction was accepted. + * + * @since 4.1.0 + */ +@Experimental +public interface SupportsPushDownVariantExtractions extends ScanBuilder { + + /** + * Pushes down variant field extractions to the data source. + *

+ * Each element in the input array represents one field extraction operation from a variant + * column. Data sources should examine each extraction and determine whether it can be + * pushed down based on the data source's capabilities (e.g., supported data types, + * path complexity, etc.). + *

+ * The return value is a boolean array of the same length as the input array, where each + * element indicates whether the corresponding extraction was accepted: + *

+ *

+ * Data sources can choose to accept all, some, or none of the extractions. Spark will + * handle any extractions that are not pushed down. + * + * @param extractions Array of variant extractions, one per field extraction operation + * @return Boolean array indicating which extractions were accepted (same length as input) + */ + boolean[] pushVariantExtractions(VariantExtraction[] extractions); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariants.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariants.java deleted file mode 100644 index d0e16e21ce766..0000000000000 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariants.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.connector.read; - -import org.apache.spark.annotation.Experimental; - -/** - * A mix-in interface for {@link Scan}. Data sources can implement this interface to - * support pushing down variant field access operations to the data source. - *

- * When variant columns are accessed with specific field extractions (e.g., variant_get), - * the optimizer can push these accesses down to the data source. The data source can then - * read only the required fields from variant columns, reducing I/O and improving performance. - *

- * The typical workflow is: - *

    - *
  1. Optimizer analyzes the query plan and identifies variant field accesses
  2. - *
  3. Optimizer calls {@link #pushVariantAccess} with the access information
  4. - *
  5. Data source validates and stores the variant access information
  6. - *
  7. Optimizer retrieves pushed information via {@link #pushedVariantAccess}
  8. - *
  9. Data source uses the information to optimize reading in {@link #readSchema()} - * and readers
  10. - *
- * - * @since 4.1.0 - */ -@Experimental -public interface SupportsPushDownVariants extends Scan { - - /** - * Pushes down variant field access information to the data source. - *

- * Implementations should validate if the variant accesses can be pushed down based on - * the data source's capabilities. If some accesses cannot be pushed down, the implementation - * can choose to: - *

    - *
  • Push down only the supported accesses and return true
  • - *
  • Reject all pushdown and return false
  • - *
- *

- * The implementation should store the variant access information that can be pushed down. - * The stored information will be retrieved later via {@link #pushedVariantAccess()}. - * - * @param variantAccessInfo Array of variant access information, one per variant column - * @return true if at least some variant accesses were pushed down, false if none were pushed - */ - boolean pushVariantAccess(VariantAccessInfo[] variantAccessInfo); - - /** - * Returns the variant access information that has been pushed down to this scan. - *

- * This method is called by the optimizer after {@link #pushVariantAccess} to retrieve - * what variant accesses were actually accepted by the data source. The optimizer uses - * this information to rewrite the query plan. - *

- * If {@link #pushVariantAccess} was not called or returned false, this should return - * an empty array. - * - * @return Array of pushed down variant access information - */ - VariantAccessInfo[] pushedVariantAccess(); -} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantAccessInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantAccessInfo.java deleted file mode 100644 index 4f61a42d05196..0000000000000 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantAccessInfo.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.connector.read; - -import java.io.Serializable; -import java.util.Objects; - -import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.types.StructType; - -/** - * Variant access information that describes how variant fields are accessed in a query. - *

- * This class captures the information needed by data sources to optimize reading variant columns. - * Instead of reading the entire variant value, the data source can read only the fields that - * are actually accessed, represented as a structured schema. - *

- * For example, if a query accesses `variant_get(v, '$.a', 'int')` and - * `variant_get(v, '$.b', 'string')`, the extracted schema would be - * `struct<0:int, 1:string>` where field ordinals correspond to the access order. - * - * @since 4.1.0 - */ -@Evolving -public final class VariantAccessInfo implements Serializable { - private final String columnName; - private final StructType extractedSchema; - - /** - * Creates variant access information for a variant column. - * - * @param columnName The name of the variant column - * @param extractedSchema The schema representing extracted fields from the variant. - * Each field represents one variant field access, with field names - * typically being ordinals (e.g., "0", "1", "2") and metadata - * containing variant-specific information like JSON path. - */ - public VariantAccessInfo(String columnName, StructType extractedSchema) { - this.columnName = Objects.requireNonNull(columnName, "columnName cannot be null"); - this.extractedSchema = - Objects.requireNonNull(extractedSchema, "extractedSchema cannot be null"); - } - - /** - * Returns the name of the variant column. - */ - public String columnName() { - return columnName; - } - - /** - * Returns the schema representing fields extracted from the variant column. - *

- * The schema structure is: - *

    - *
  • Field names: Typically ordinals ("0", "1", "2", ...) representing access order
  • - *
  • Field types: The target data type for each field extraction
  • - *
  • Field metadata: Contains variant-specific information such as JSON path, - * timezone, and error handling mode
  • - *
- *

- * Data sources should use this schema to determine what fields to extract from the variant - * and what types they should be converted to. - */ - public StructType extractedSchema() { - return extractedSchema; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - VariantAccessInfo that = (VariantAccessInfo) o; - return columnName.equals(that.columnName) && - extractedSchema.equals(that.extractedSchema); - } - - @Override - public int hashCode() { - return Objects.hash(columnName, extractedSchema); - } - - @Override - public String toString() { - return "VariantAccessInfo{" + - "columnName='" + columnName + '\'' + - ", extractedSchema=" + extractedSchema + - '}'; - } -} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantExtraction.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantExtraction.java new file mode 100644 index 0000000000000..64987299934a2 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantExtraction.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.read; + +import java.io.Serializable; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Metadata; + +/** + * Variant extraction information that describes a single field extraction from a variant column. + *

+ * This interface captures the information needed by data sources to optimize reading variant + * columns. Each instance represents one field extraction operation (e.g., from variant_get or + * try_variant_get). + *

+ * For example, if a query contains `variant_get(v, '$.a', 'int')`, this would be represented + * as a VariantExtraction with columnName=["v"], path="$.a", and expectedDataType=IntegerType. + * + * @since 4.1.0 + */ +@Experimental +public interface VariantExtraction extends Serializable { + /** + * Returns the path to the variant column. For top-level variant columns, this is a single + * element array containing the column name. For nested variant columns within structs, + * this is an array representing the path (e.g., ["structCol", "innerStruct", "variantCol"]). + */ + String[] columnName(); + + /** + * Returns the expected data type for the extracted value. + * This is the target type specified in variant_get (e.g., IntegerType, StringType). + */ + DataType expectedDataType(); + + /** + * Returns the metadata associated with this variant extraction. + * This may include additional information needed by the data source: + * - "path": the extraction path from variant_get or try_variant_get. + * This follows JSON path syntax (e.g., "$.a", "$.b.c", "$[0]"). + * - "failOnError": whether the extraction to expected data type should throw an exception + * or return null if the cast fails. + * - "timeZoneId": a string identifier of a time zone. It is required by timestamp-related casts. + * + */ + Metadata metadata(); +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/VariantExtractionImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/VariantExtractionImpl.scala new file mode 100644 index 0000000000000..87db41a3217e6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/VariantExtractionImpl.scala @@ -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.spark.sql.internal.connector + +import org.apache.spark.sql.connector.read.VariantExtraction +import org.apache.spark.sql.types.{DataType, Metadata} + +/** + * Implementation of [[VariantExtraction]]. + * + * @param columnName Path to the variant column (e.g., Array("v") for top-level, + * Array("struct1", "v") for nested) + * @param metadata The metadata for extraction including JSON path, failOnError, and timeZoneId + * @param expectedDataType The expected data type for the extracted value + */ +case class VariantExtractionImpl( + columnName: Array[String], + metadata: Metadata, + expectedDataType: DataType) extends VariantExtraction { + + require(columnName != null, "columnName cannot be null") + require(metadata != null, "metadata cannot be null") + require(expectedDataType != null, "expectedDataType cannot be null") + require(columnName.nonEmpty, "columnName cannot be empty") + + override def toString: String = { + s"VariantExtraction{columnName=${columnName.mkString("[", ", ", "]")}, " + + s"metadata='$metadata', expectedDataType=$expectedDataType}" + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala index 2cf1a5e9b8cdc..b0b20d08dccbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala @@ -26,10 +26,8 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, Subquery} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns -import org.apache.spark.sql.connector.read.{SupportsPushDownVariants, VariantAccessInfo} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -225,8 +223,11 @@ class VariantInRelation { case Some(variants) => variants.get(path) match { case Some(fields) => + // Accessing the full variant value addField(fields, RequestedVariantField.fullVariant) case _ => + // Accessing the struct containing a variant. + // This variant is not eligible for push down. // Remove non-eligible variants. variants.filterInPlace { case (key, _) => !key.startsWith(path) } } @@ -281,11 +282,6 @@ object PushVariantIntoScan extends Rule[LogicalPlan] { relation @ LogicalRelationWithTable( hadoopFsRelation@HadoopFsRelation(_, _, _, _, _: ParquetFileFormat, _), _)) => rewritePlan(p, projectList, filters, relation, hadoopFsRelation) - - case p@PhysicalOperation(projectList, filters, - scanRelation @ DataSourceV2ScanRelation( - relation, scan: SupportsPushDownVariants, output, _, _)) => - rewritePlanV2(p, projectList, filters, scanRelation, scan) } } @@ -333,112 +329,10 @@ object PushVariantIntoScan extends Rule[LogicalPlan] { hadoopFsRelation.sparkSession) val newRelation = relation.copy(relation = newHadoopFsRelation, output = newOutput.toIndexedSeq) - buildFilterAndProject(newRelation, projectList, filters, variants, attributeMap) - } - - // DataSource V2 rewrite method using SupportsPushDownVariants API - // Key differences from V1 implementation: - // 1. V2 uses DataSourceV2ScanRelation instead of LogicalRelation - // 2. Uses SupportsPushDownVariants API instead of directly manipulating scan - // 3. Schema is already resolved in scanRelation.output (no need for relation.resolve()) - // 4. Scan rebuilding is handled by the scan implementation via the API - // Data sources like Delta and Iceberg can implement this API to support variant pushdown. - private def rewritePlanV2( - originalPlan: LogicalPlan, - projectList: Seq[NamedExpression], - filters: Seq[Expression], - scanRelation: DataSourceV2ScanRelation, - scan: SupportsPushDownVariants): LogicalPlan = { - val variants = new VariantInRelation - - // Extract schema attributes from V2 scan relation - val schemaAttributes = scanRelation.output - - // Construct schema for default value resolution - val structSchema = StructType(schemaAttributes.map(a => - StructField(a.name, a.dataType, a.nullable, a.metadata))) - - val defaultValues = ResolveDefaultColumns.existenceDefaultValues(structSchema) - - // Add variant fields from the V2 scan schema - for ((a, defaultValue) <- schemaAttributes.zip(defaultValues)) { - variants.addVariantFields(a.exprId, a.dataType, defaultValue, Nil) - } - if (variants.mapping.isEmpty) return originalPlan - - // Collect requested fields from project list and filters - projectList.foreach(variants.collectRequestedFields) - filters.foreach(variants.collectRequestedFields) - - // If no variant columns remain after collection, return original plan - if (variants.mapping.forall(_._2.isEmpty)) return originalPlan - - // Build VariantAccessInfo array for the API - val variantAccessInfoArray = schemaAttributes.flatMap { attr => - variants.mapping.get(attr.exprId).flatMap(_.get(Nil)).map { fields => - // Build extracted schema for this variant column - val extractedFields = fields.toArray.sortBy(_._2).map { case (field, ordinal) => - StructField(ordinal.toString, field.targetType, metadata = field.path.toMetadata) - } - val extractedSchema = if (extractedFields.isEmpty) { - // Add placeholder field to avoid empty struct - val placeholder = VariantMetadata("$.__placeholder_field__", - failOnError = false, timeZoneId = "UTC") - StructType(Array(StructField("0", BooleanType, metadata = placeholder.toMetadata))) - } else { - StructType(extractedFields) - } - new VariantAccessInfo(attr.name, extractedSchema) - } - }.toArray - - // Call the API to push down variant access - if (variantAccessInfoArray.isEmpty) return originalPlan - - val pushed = scan.pushVariantAccess(variantAccessInfoArray) - if (!pushed) return originalPlan - - // Get what was actually pushed - val pushedVariantAccess = scan.pushedVariantAccess() - if (pushedVariantAccess.isEmpty) return originalPlan - - // Build new attribute mapping based on pushed variant access - val pushedColumnNames = pushedVariantAccess.map(_.columnName()).toSet - val attributeMap = schemaAttributes.map { a => - if (pushedColumnNames.contains(a.name) && variants.mapping.get(a.exprId).exists(_.nonEmpty)) { - val newType = variants.rewriteType(a.exprId, a.dataType, Nil) - val newAttr = AttributeReference(a.name, newType, a.nullable, a.metadata)( - qualifier = a.qualifier) - (a.exprId, newAttr) - } else { - (a.exprId, a) - } - }.toMap - - val newOutput = scanRelation.output.map(a => attributeMap.getOrElse(a.exprId, a)) - - // The scan implementation should have updated its readSchema() based on the pushed info - // We just need to create a new scan relation with the updated output - val newScanRelation = scanRelation.copy( - output = newOutput - ) - - buildFilterAndProject(newScanRelation, projectList, filters, variants, attributeMap) - } - - /** - * Build the final Project(Filter(relation)) plan with rewritten expressions. - */ - private def buildFilterAndProject( - relation: LogicalPlan, - projectList: Seq[NamedExpression], - filters: Seq[Expression], - variants: VariantInRelation, - attributeMap: Map[ExprId, AttributeReference]): LogicalPlan = { val withFilter = if (filters.nonEmpty) { - Filter(filters.map(variants.rewriteExpr(_, attributeMap)).reduce(And), relation) + Filter(filters.map(variants.rewriteExpr(_, attributeMap)).reduce(And), newRelation) } else { - relation + newRelation } val newProjectList = projectList.map { e => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 31a98e1ff96cb..adfd5ceacd675 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -21,8 +21,9 @@ import java.util.Locale import scala.collection.mutable +import org.apache.spark.SparkException import org.apache.spark.internal.LogKeys.{AGGREGATE_FUNCTIONS, COLUMN_NAMES, GROUP_BY_EXPRS, JOIN_CONDITION, JOIN_TYPE, POST_SCAN_FILTERS, PUSHED_FILTERS, RELATION_NAME, RELATION_OUTPUT} -import org.apache.spark.sql.catalyst.expressions.{aggregate, Alias, And, Attribute, AttributeMap, AttributeReference, AttributeSet, Cast, Expression, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{aggregate, Alias, And, Attribute, AttributeMap, AttributeReference, AttributeSet, Cast, Expression, ExprId, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.CollapseProject import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ScanOperation} @@ -32,10 +33,11 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.connector.expressions.{SortOrder => V2SortOrder} import org.apache.spark.sql.connector.expressions.aggregate.{Aggregation, Avg, Count, CountStar, Max, Min, Sum} import org.apache.spark.sql.connector.expressions.filter.Predicate -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownJoin, V1Scan} -import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownJoin, SupportsPushDownVariantExtractions, V1Scan, VariantExtraction} +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, VariantInRelation} +import org.apache.spark.sql.internal.connector.VariantExtractionImpl import org.apache.spark.sql.sources -import org.apache.spark.sql.types.{DataType, DecimalType, IntegerType, StructType} +import org.apache.spark.sql.types.{DataType, DecimalType, IntegerType, StructField, StructType} import org.apache.spark.sql.util.SchemaUtils._ import org.apache.spark.util.ArrayImplicits._ @@ -49,9 +51,11 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper { pushDownFilters, pushDownJoin, pushDownAggregates, + pushDownVariants, pushDownLimitAndOffset, buildScanWithPushedAggregate, buildScanWithPushedJoin, + buildScanWithPushedVariants, pruneColumns) pushdownRules.foldLeft(plan) { (newPlan, pushDownRule) => @@ -318,6 +322,139 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper { case agg: Aggregate => rewriteAggregate(agg) } + def pushDownVariants(plan: LogicalPlan): LogicalPlan = plan.transformDown { + case p@PhysicalOperation(projectList, filters, sHolder @ ScanBuilderHolder(_, _, + builder: SupportsPushDownVariantExtractions)) + if conf.getConf(org.apache.spark.sql.internal.SQLConf.PUSH_VARIANT_INTO_SCAN) => + pushVariantExtractions(p, projectList, filters, sHolder, builder) + } + + /** + * Converts an ordinal path to a field name path. + * + * @param structType The top-level struct type + * @param ordinals The ordinal path (e.g., [1, 1] for nested.field) + * @return The field name path (e.g., ["nested", "field"]) + */ + private def getColumnName(structType: StructType, ordinals: Seq[Int]): Seq[String] = { + ordinals match { + case Seq() => + // Base case: no more ordinals + Seq.empty + case ordinal +: rest => + // Get the field at this ordinal + val field = structType.fields(ordinal) + if (rest.isEmpty) { + // Last ordinal in the path + Seq(field.name) + } else { + // Recurse into nested struct + field.dataType match { + case nestedStruct: StructType => + field.name +: getColumnName(nestedStruct, rest) + case _ => + throw SparkException.internalError( + s"Expected StructType at field '${field.name}' but got ${field.dataType}") + } + } + } + } + + private def pushVariantExtractions( + originalPlan: LogicalPlan, + projectList: Seq[NamedExpression], + filters: Seq[Expression], + sHolder: ScanBuilderHolder, + builder: SupportsPushDownVariantExtractions): LogicalPlan = { + val variants = new VariantInRelation + + // Extract schema attributes from scan builder holder + val schemaAttributes = sHolder.output + + // Construct schema for default value resolution + val structSchema = StructType(schemaAttributes.map(a => + StructField(a.name, a.dataType, a.nullable, a.metadata))) + + val defaultValues = org.apache.spark.sql.catalyst.util.ResolveDefaultColumns. + existenceDefaultValues(structSchema) + + // Add variant fields from the V2 scan schema + for ((a, defaultValue) <- schemaAttributes.zip(defaultValues)) { + variants.addVariantFields(a.exprId, a.dataType, defaultValue, Nil) + } + if (variants.mapping.isEmpty) return originalPlan + + // Collect requested fields from project list and filters + projectList.foreach(variants.collectRequestedFields) + filters.foreach(variants.collectRequestedFields) + + // If no variant columns remain after collection, return original plan + if (variants.mapping.forall(_._2.isEmpty)) return originalPlan + + // Build individual VariantExtraction for each field access + // Track which extraction corresponds to which (attr, field, ordinal) + val extractionInfo = schemaAttributes.flatMap { topAttr => + val variantFields = variants.mapping.get(topAttr.exprId) + if (variantFields.isEmpty || variantFields.get.isEmpty) { + // No variant fields for this attribute + Seq.empty + } else { + variantFields.get.toSeq.flatMap { case (pathToVariant, fields) => + val columnName = if (pathToVariant.isEmpty) { + Seq(topAttr.name) + } else { + Seq(topAttr.name) ++ + getColumnName(topAttr.dataType.asInstanceOf[StructType], pathToVariant) + } + fields.toArray.sortBy(_._2).map { case (field, ordinal) => + val extraction = new VariantExtractionImpl( + columnName.toArray, + field.path.toMetadata, + field.targetType + ) + (extraction, topAttr, field, ordinal) + } + } + } + } + + // Call the API to push down variant extractions + if (extractionInfo.isEmpty) return originalPlan + + val extractions: Array[VariantExtraction] = extractionInfo.map(_._1).toArray + val pushedResults = builder.pushVariantExtractions(extractions) + + // Filter to only the accepted extractions + val acceptedExtractions = extractionInfo.zip(pushedResults).filter(_._2).map(_._1) + if (acceptedExtractions.isEmpty) return originalPlan + + // Group accepted extractions by attribute to rebuild the struct schemas + val extractionsByAttr = acceptedExtractions.groupBy(_._2) + val pushedColumnNames = extractionsByAttr.keys.map(_.name).toSet + + // Build new attribute mapping based on pushed variant extractions + val attributeMap = schemaAttributes.map { a => + if (pushedColumnNames.contains(a.name) && variants.mapping.get(a.exprId).exists(_.nonEmpty)) { + val newType = variants.rewriteType(a.exprId, a.dataType, Nil) + val newAttr = AttributeReference(a.name, newType, a.nullable, a.metadata)( + qualifier = a.qualifier) + (a.exprId, newAttr) + } else { + (a.exprId, a.asInstanceOf[AttributeReference]) + } + }.toMap + + val newOutput = sHolder.output.map(a => attributeMap.getOrElse(a.exprId, a)) + + // Store the transformation info on the holder for later use + sHolder.pushedVariants = Some(variants) + sHolder.pushedVariantAttributeMap = attributeMap + sHolder.output = newOutput + + // Return the original plan unchanged - transformation happens in buildScanWithPushedVariants + originalPlan + } + private def rewriteAggregate(agg: Aggregate): LogicalPlan = agg.child match { case PhysicalOperation(project, Nil, holder @ ScanBuilderHolder(_, _, r: SupportsPushDownAggregates)) if CollapseProject.canCollapseExpressions( @@ -589,6 +726,48 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper { Project(projectList, scanRelation) } + def buildScanWithPushedVariants(plan: LogicalPlan): LogicalPlan = plan.transform { + case p@PhysicalOperation(projectList, filters, holder: ScanBuilderHolder) + if holder.pushedVariants.isDefined => + val variants = holder.pushedVariants.get + val attributeMap = holder.pushedVariantAttributeMap + + // Build the scan + val scan = holder.builder.build() + val realOutput = toAttributes(scan.readSchema()) + val wrappedScan = getWrappedScan(scan, holder) + val scanRelation = DataSourceV2ScanRelation(holder.relation, wrappedScan, realOutput) + + // Create projection to map real output to expected output (with transformed types) + val outputProjection = realOutput.zip(holder.output).map { case (realAttr, expectedAttr) => + Alias(realAttr, expectedAttr.name)(expectedAttr.exprId) + } + + // Rewrite filter expressions using the variant transformation + val rewrittenFilters = if (filters.nonEmpty) { + val rewrittenFilterExprs = filters.map(variants.rewriteExpr(_, attributeMap)) + Some(rewrittenFilterExprs.reduce(And)) + } else { + None + } + + // Rewrite project list expressions using the variant transformation + val rewrittenProjectList = projectList.map { e => + val rewritten = variants.rewriteExpr(e, attributeMap) + rewritten match { + case n: NamedExpression => n + // When the variant column is directly selected, we replace the attribute + // reference with a struct access, which is not a NamedExpression. Wrap it with Alias. + case _ => Alias(rewritten, e.name)(e.exprId, e.qualifier) + } + } + + // Build the plan: Project(outputProjection) -> [Filter?] -> scanRelation + val withProjection = Project(outputProjection, scanRelation) + val withFilter = rewrittenFilters.map(Filter(_, withProjection)).getOrElse(withProjection) + Project(rewrittenProjectList, withFilter) + } + def pruneColumns(plan: LogicalPlan): LogicalPlan = plan.transform { case ScanOperation(project, filtersStayUp, filtersPushDown, sHolder: ScanBuilderHolder) => // column pruning @@ -834,6 +1013,10 @@ case class ScanBuilderHolder( var joinedRelationsPushedDownOperators: Seq[PushedDownOperators] = Seq.empty[PushedDownOperators] var pushedJoinOutputMap: AttributeMap[Expression] = AttributeMap.empty[Expression] + + var pushedVariantAttributeMap: Map[ExprId, AttributeReference] = Map.empty + + var pushedVariants: Option[VariantInRelation] = None } // A wrapper for v1 scan to carry the translated filters and the handled ones, along with diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index d347cb04f0bcf..5a427aad5f895 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -25,13 +25,13 @@ import org.apache.parquet.hadoop.ParquetInputFormat import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.expressions.aggregate.Aggregation -import org.apache.spark.sql.connector.read.{PartitionReaderFactory, SupportsPushDownVariants, VariantAccessInfo} -import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, PartitioningAwareFileIndex} +import org.apache.spark.sql.connector.read.{PartitionReaderFactory, VariantExtraction} +import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, PartitioningAwareFileIndex, VariantMetadata} import org.apache.spark.sql.execution.datasources.parquet.{ParquetOptions, ParquetReadSupport, ParquetWriteSupport} import org.apache.spark.sql.execution.datasources.v2.FileScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{BooleanType, DataType, StructField, StructType, VariantType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SerializableConfiguration @@ -48,8 +48,7 @@ case class ParquetScan( pushedAggregate: Option[Aggregation] = None, partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty, - pushedVariantAccessInfo: Array[VariantAccessInfo] = Array.empty) extends FileScan - with SupportsPushDownVariants { + pushedVariantExtractions: Array[VariantExtraction] = Array.empty) extends FileScan { override def isSplitable(path: Path): Boolean = { // If aggregate is pushed down, only the file footer will be read once, // so file should not be split across multiple tasks. @@ -58,20 +57,57 @@ case class ParquetScan( // Build transformed schema if variant pushdown is active private def effectiveReadDataSchema: StructType = { - if (_pushedVariantAccess.isEmpty) { + if (pushedVariantExtractions.isEmpty) { readDataSchema } else { - // Build a mapping from column name to extracted schema - val variantSchemaMap = _pushedVariantAccess.map(info => - info.columnName() -> info.extractedSchema()).toMap - - // Transform the read data schema by replacing variant columns with their extracted schemas - StructType(readDataSchema.map { field => - variantSchemaMap.get(field.name) match { - case Some(extractedSchema) => field.copy(dataType = extractedSchema) - case None => field + rewriteVariantPushdownSchema(readDataSchema) + } + } + + private def rewriteVariantPushdownSchema(schema: StructType): StructType = { + // Group extractions by column name and build extracted schemas + val variantSchemaMap: Map[Seq[String], StructType] = pushedVariantExtractions + .groupBy(e => e.columnName().toSeq) + .map { case (colName, extractions) => + // Build struct schema with ordinal-named fields for each extraction + var fields = extractions.zipWithIndex.map { case (extraction, idx) => + // Attach VariantMetadata so Parquet reader knows this is a variant extraction + StructField(idx.toString, extraction.expectedDataType(), nullable = true, + extraction.metadata()) + } + + // Avoid producing an empty struct of requested fields. This happens + // if the variant is not used, or only used in `IsNotNull/IsNull` expressions. + // The value of the placeholder field doesn't matter. + if (fields.size == 1 && fields.head.dataType.isInstanceOf[VariantType]) { + val placeholder = VariantMetadata("$.__placeholder_field__", + failOnError = false, timeZoneId = "UTC") + fields = Array(StructField("0", BooleanType, + metadata = placeholder.toMetadata)) } - }) + + colName -> StructType(fields) + }.toMap + + rewriteType(schema, Seq.empty, variantSchemaMap).asInstanceOf[StructType] + } + + private def rewriteType( + dataType: DataType, + path: Seq[String], + mapping: Map[Seq[String], StructType]): DataType = { + dataType match { + case structType: StructType if !VariantMetadata.isVariantStruct(structType) => + val fields = structType.fields.map { field => + mapping.get(path :+ field.name) match { + case Some(extractedSchema) => + field.copy(dataType = extractedSchema) + case None => + field.copy(dataType = rewriteType(field.dataType, path :+ field.name, mapping)) + } + } + StructType(fields) + case otherType => otherType } } @@ -84,38 +120,14 @@ case class ParquetScan( // super.readSchema() combines readDataSchema + readPartitionSchema // Apply variant transformation if variant pushdown is active val baseSchema = super.readSchema() - if (_pushedVariantAccess.isEmpty) { + if (pushedVariantExtractions.isEmpty) { baseSchema } else { - val variantSchemaMap = _pushedVariantAccess.map(info => - info.columnName() -> info.extractedSchema()).toMap - StructType(baseSchema.map { field => - variantSchemaMap.get(field.name) match { - case Some(extractedSchema) => field.copy(dataType = extractedSchema) - case None => field - } - }) + rewriteVariantPushdownSchema(baseSchema) } } } - // SupportsPushDownVariants API implementation - private var _pushedVariantAccess: Array[VariantAccessInfo] = pushedVariantAccessInfo - - override def pushVariantAccess(variantAccessInfo: Array[VariantAccessInfo]): Boolean = { - // Parquet supports variant pushdown for all variant accesses - if (variantAccessInfo.nonEmpty) { - _pushedVariantAccess = variantAccessInfo - true - } else { - false - } - } - - override def pushedVariantAccess(): Array[VariantAccessInfo] = { - _pushedVariantAccess - } - override def createReaderFactory(): PartitionReaderFactory = { val effectiveSchema = effectiveReadDataSchema val readDataSchemaAsJson = effectiveSchema.json @@ -171,8 +183,8 @@ case class ParquetScan( pushedAggregate.isEmpty && p.pushedAggregate.isEmpty } val pushedVariantEqual = - java.util.Arrays.equals(_pushedVariantAccess.asInstanceOf[Array[Object]], - p._pushedVariantAccess.asInstanceOf[Array[Object]]) + java.util.Arrays.equals(pushedVariantExtractions.asInstanceOf[Array[Object]], + p.pushedVariantExtractions.asInstanceOf[Array[Object]]) super.equals(p) && dataSchema == p.dataSchema && options == p.options && equivalentFilters(pushedFilters, p.pushedFilters) && pushedDownAggEqual && pushedVariantEqual @@ -189,15 +201,17 @@ case class ParquetScan( } override def getMetaData(): Map[String, String] = { - val variantAccessStr = if (_pushedVariantAccess.nonEmpty) { - _pushedVariantAccess.map(info => - s"${info.columnName()}->${info.extractedSchema()}").mkString("[", ", ", "]") + val variantExtractionStr = if (pushedVariantExtractions.nonEmpty) { + pushedVariantExtractions.map { extraction => + val colName = extraction.columnName().mkString(".") + s"$colName:${extraction.metadata()}:${extraction.expectedDataType()}" + }.mkString("[", ", ", "]") } else { "[]" } super.getMetaData() ++ Map("PushedFilters" -> seqToString(pushedFilters.toImmutableArraySeq)) ++ Map("PushedAggregation" -> pushedAggregationsStr) ++ Map("PushedGroupBy" -> pushedGroupByStr) ++ - Map("PushedVariantAccess" -> variantAccessStr) + Map("PushedVariantExtractions" -> variantExtractionStr) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala index 01367675e65b9..94da53f229349 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala @@ -22,7 +22,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec import org.apache.spark.sql.connector.expressions.aggregate.Aggregation -import org.apache.spark.sql.connector.read.SupportsPushDownAggregates +import org.apache.spark.sql.connector.read.{SupportsPushDownAggregates, SupportsPushDownVariantExtractions, VariantExtraction} import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, PartitioningAwareFileIndex} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, SparkToParquetSchemaConverter} import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder @@ -39,7 +39,8 @@ case class ParquetScanBuilder( dataSchema: StructType, options: CaseInsensitiveStringMap) extends FileScanBuilder(sparkSession, fileIndex, dataSchema) - with SupportsPushDownAggregates { + with SupportsPushDownAggregates + with SupportsPushDownVariantExtractions { lazy val hadoopConf = { val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap // Hadoop Configurations are case sensitive. @@ -50,6 +51,8 @@ case class ParquetScanBuilder( private var pushedAggregations = Option.empty[Aggregation] + private var pushedVariantExtractions = Array.empty[VariantExtraction] + override protected val supportsNestedSchemaPruning: Boolean = true override def pushDataFilters(dataFilters: Array[Filter]): Array[Filter] = { @@ -99,6 +102,14 @@ case class ParquetScanBuilder( } } + // SupportsPushDownVariantExtractions API implementation + override def pushVariantExtractions(extractions: Array[VariantExtraction]): Array[Boolean] = { + // Parquet supports variant pushdown for all variant extractions + pushedVariantExtractions = extractions + // Return true for all extractions (Parquet can handle all of them) + Array.fill(extractions.length)(true) + } + override def build(): ParquetScan = { // the `finalSchema` is either pruned in pushAggregation (if aggregates are // pushed down), or pruned in readDataSchema() (in regular column pruning). These @@ -108,6 +119,6 @@ case class ParquetScanBuilder( } ParquetScan(sparkSession, hadoopConf, fileIndex, dataSchema, finalSchema, readPartitionSchema(), pushedDataFilters, options, pushedAggregations, - partitionFilters, dataFilters) + partitionFilters, dataFilters, pushedVariantExtractions) } } From e3b4066acd7321335f95dac9191a42abc55e760a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 10 Dec 2025 18:02:08 -0800 Subject: [PATCH 286/400] [SPARK-54679][SQL] Rename `spark.sql.(xml.legacyXMLParser.enabled -> legacy.useLegacyXMLParser)` ### What changes were proposed in this pull request? This PR aims to use `spark.sql.legacy` namespace instead of adding a new `spark.sql.xml` namespace for a single legacy behavior. ### Why are the changes needed? For simplify by reusing the existing `spark.sql.legacy` namespace. ### Does this PR introduce _any_ user-facing change? No. This configuration is added at Apache Spark 4.1.0 via the following. - https://github.com/apache/spark/pull/51287 ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53433 from dongjoon-hyun/SPARK-54679. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 2eabf4d08b1561d67d9526e0400a09a34d8a58df) Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/execution/datasources/xml/XmlInferSchemaSuite.scala | 2 +- .../sql/execution/datasources/xml/XmlPartitioningSuite.scala | 3 ++- .../apache/spark/sql/execution/datasources/xml/XmlSuite.scala | 2 +- .../spark/sql/execution/datasources/xml/XmlVariantSuite.scala | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a4cee92e38c2f..c4c46bbe67f1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6590,7 +6590,7 @@ object SQLConf { .createWithDefault(false) val LEGACY_XML_PARSER_ENABLED = { - buildConf("spark.sql.xml.legacyXMLParser.enabled") + buildConf("spark.sql.legacy.useLegacyXMLParser") .internal() .doc( "When set to true, use the legacy XML parser for parsing XML files. " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala index 9f36cb3ad2cde..05e5d6d17628d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala @@ -46,7 +46,7 @@ class XmlInferSchemaSuite protected val legacyParserEnabled: Boolean = false override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.xml.legacyXMLParser.enabled", legacyParserEnabled.toString) + .set(SQLConf.LEGACY_XML_PARSER_ENABLED, legacyParserEnabled) private val baseOptions = Map("rowTag" -> "ROW") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala index 22bb66daf3134..2a5a36c74a281 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.matchers.should.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.SQLConf /** * Tests various cases of partition size, compression. @@ -33,7 +34,7 @@ class XmlPartitioningSuite extends SparkFunSuite with Matchers with BeforeAndAft .master("local[2]") .appName("XmlPartitioningSuite") .config("spark.hadoop.fs.local.block.size", blockSize) - .config("spark.sql.xml.legacyXMLParser.enabled", legacyParserEnabled) + .config(SQLConf.LEGACY_XML_PARSER_ENABLED.key, legacyParserEnabled) .getOrCreate() try { val fileName = s"test-data/xml-resources/fias_house${if (large) ".large" else ""}.xml$suffix" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala index ce5194864c10d..06845cdf8120e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala @@ -63,7 +63,7 @@ class XmlSuite protected val legacyParserEnabled: Boolean = false override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.xml.legacyXMLParser.enabled", legacyParserEnabled.toString) + .set(SQLConf.LEGACY_XML_PARSER_ENABLED, legacyParserEnabled) protected val resDir = "test-data/xml-resources/" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala index 15713e759b81b..5738cd2a99272 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlVariantSuite.scala @@ -34,7 +34,7 @@ class XmlVariantSuite extends QueryTest with SharedSparkSession with TestXmlData protected val legacyParserEnabled: Boolean = false override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.xml.legacyXMLParser.enabled", legacyParserEnabled.toString) + .set(SQLConf.LEGACY_XML_PARSER_ENABLED, legacyParserEnabled) private val baseOptions = Map("rowTag" -> "ROW", "valueTag" -> "_VALUE", "attributePrefix" -> "_") From c7ad2bf3d766eef4956ff270ff70f762445bb9a6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 10 Dec 2025 18:13:56 -0800 Subject: [PATCH 287/400] [SPARK-53482][SQL][FOLLOWUP] Rename `spark.sql.merge(.nested.type.coercion.enabled -> NestedTypeCoercion.enabled)` ### What changes were proposed in this pull request? This PR aims to rename `spark.sql.merge.nested.type.coercion.enabled` to follow the Apache Spark convention. ``` - spark.sql.merge.nested.type.coercion.enabled + spark.sql.mergeNestedTypeCoercion.enabled ``` ### Why are the changes needed? To avoid to create many naming spaces. ### Does this PR introduce _any_ user-facing change? Yes, but this is not released as 4.1.0. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53434 from dongjoon-hyun/SPARK-53482. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit f43d5cea838a4fb2f08655e812a49fc9639eeb2f) Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c4c46bbe67f1e..55c5eab5dda0b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6612,7 +6612,7 @@ object SQLConf { .createWithDefault(true) val MERGE_INTO_NESTED_TYPE_COERCION_ENABLED = - buildConf("spark.sql.merge.nested.type.coercion.enabled") + buildConf("spark.sql.mergeNestedTypeCoercion.enabled") .internal() .doc("If enabled, allow MERGE INTO to coerce source nested types if they have less" + "nested fields than the target table's nested types. This is experimental and" + From ebe1fcced0c522e02a9a0af65669584f0c720fc8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 10 Dec 2025 19:19:45 -0800 Subject: [PATCH 288/400] [SPARK-52837][PYTHON][FOLLOWUP] Add `versionadded` to `TimeType` doc ### What changes were proposed in this pull request? This PR aims to add `versionadded` to `TimeType`. ### Why are the changes needed? For `Scala/Java` documentation, we have `since` information. https://github.com/apache/spark/blob/22e26598785e11303409f43fc08ce627053dadb4/sql/api/src/main/scala/org/apache/spark/sql/types/TimeType.scala#L31-L34 We had better provide a correct information for the newly added `TimeType` in Python documentation. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53436 from dongjoon-hyun/SPARK-52837. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit d80a0f382d406808c18c0baa44d9ecba623ba701) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/types.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 95307ea3859c7..0504999ac65aa 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -403,7 +403,11 @@ class AnyTimeType(DatetimeType): class TimeType(AnyTimeType): - """Time (datetime.time) data type.""" + """ + Time (datetime.time) data type. + + .. versionadded:: 4.1.0 + """ def __init__(self, precision: int = 6): self.precision = precision From ed92a5cc3ea21b9be442b642621e2b582013161a Mon Sep 17 00:00:00 2001 From: David Milicevic Date: Wed, 10 Dec 2025 19:29:17 -0800 Subject: [PATCH 289/400] [SPARK-54609][SQL] Disable TIME type by default Introducing a new SQL config for TIME type: `spark.sql.timeType.enabled`. The default value is `false` and it is enabled only in tests. TIME data type support is not complete, so we need to guard it before it is completed, especially ahead of Spark 4.1 release. No. Need to add tests for disabled config. No. Closes #53344 from davidm-db/davidm-db/time-config. Lead-authored-by: David Milicevic Co-authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit 18a9435f28e602e6adfcc7f484917f8b5bfb6b48) Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 6 +++ .../sql/catalyst/CatalystTypeConverters.scala | 5 ++- .../spark/sql/catalyst/expressions/Cast.scala | 8 +++- .../expressions/timeExpressions.scala | 38 +++++++++++++------ .../sql/errors/QueryCompilationErrors.scala | 6 +++ .../apache/spark/sql/internal/SQLConf.scala | 9 +++++ .../execution/SparkConnectPlanExecution.scala | 7 +++- .../datasources/DataSourceUtils.scala | 3 ++ sql/gen-sql-functions-docs.py | 2 + 9 files changed, 69 insertions(+), 15 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index b79e44a7984a0..dfaf8425a1a0c 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -7141,6 +7141,12 @@ ], "sqlState" : "0A001" }, + "UNSUPPORTED_TIME_TYPE" : { + "message" : [ + "The data type TIME is not supported." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_TYPED_LITERAL" : { "message" : [ "Literals of the type are not supported. Supported types are ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 53803192cb3d9..a90720ac5108e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -25,12 +25,11 @@ import java.time.{Duration, Instant, LocalDate, LocalDateTime, LocalTime, Period import java.util.{Map => JavaMap} import javax.annotation.Nullable -import scala.language.existentials - import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DayTimeIntervalType._ @@ -79,6 +78,8 @@ object CatalystTypeConverters { new GeometryConverter(g) case DateType if SQLConf.get.datetimeJava8ApiEnabled => LocalDateConverter case DateType => DateConverter + case _: TimeType if !SQLConf.get.isTimeTypeEnabled => + QueryCompilationErrors.unsupportedTimeTypeError() case _: TimeType => TimeConverter case TimestampType if SQLConf.get.datetimeJava8ApiEnabled => InstantConverter case TimestampType => TimestampConverter diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1162a5394221c..03b6a452fe7da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.catalyst.util.IntervalUtils.{dayTimeIntervalToByte, dayTimeIntervalToDecimal, dayTimeIntervalToInt, dayTimeIntervalToLong, dayTimeIntervalToShort, yearMonthIntervalToByte, yearMonthIntervalToInt, yearMonthIntervalToShort} -import org.apache.spark.sql.errors.{QueryErrorsBase, QueryExecutionErrors} +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{GeographyVal, UTF8String, VariantVal} @@ -602,6 +602,12 @@ case class Cast( } override def checkInputDataTypes(): TypeCheckResult = { + dataType match { + // If the cast is to a TIME type, first check if TIME type is enabled. + case _: TimeType if !SQLConf.get.isTimeTypeEnabled => + throw QueryCompilationErrors.unsupportedTimeTypeError() + case _ => + } val canCast = evalMode match { case EvalMode.LEGACY => Cast.canCast(child.dataType, dataType) case EvalMode.ANSI => Cast.canAnsiCast(child.dataType, dataType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala index ff088876969bd..692dd5b1f3987 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala @@ -32,11 +32,22 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.TimeFormatter import org.apache.spark.sql.catalyst.util.TypeUtils.ordinalNumber import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types.{AbstractDataType, AnyTimeType, ByteType, DataType, DayTimeIntervalType, DecimalType, IntegerType, LongType, ObjectType, TimeType} import org.apache.spark.sql.types.DayTimeIntervalType.{HOUR, SECOND} import org.apache.spark.unsafe.types.UTF8String +trait TimeExpression extends Expression { + override def checkInputDataTypes(): TypeCheckResult = { + if (SQLConf.get.isTimeTypeEnabled) { + super.checkInputDataTypes() + } else { + throw QueryCompilationErrors.unsupportedTimeTypeError() + } + } +} + /** * Parses a column to a time based on the given format. */ @@ -64,7 +75,7 @@ import org.apache.spark.unsafe.types.UTF8String since = "4.1.0") // scalastyle:on line.size.limit case class ToTime(str: Expression, format: Option[Expression]) - extends RuntimeReplaceable with ExpectsInputTypes { + extends RuntimeReplaceable with ExpectsInputTypes with TimeExpression { def this(str: Expression, format: Expression) = this(str, Option(format)) def this(str: Expression) = this(str, None) @@ -200,7 +211,7 @@ object TryToTimeExpressionBuilder extends ExpressionBuilder { // scalastyle:on line.size.limit case class MinutesOfTime(child: Expression) extends RuntimeReplaceable - with ExpectsInputTypes { + with ExpectsInputTypes with TimeExpression { override def replacement: Expression = StaticInvoke( classOf[DateTimeUtils.type], @@ -259,7 +270,7 @@ object MinuteExpressionBuilder extends ExpressionBuilder { case class HoursOfTime(child: Expression) extends RuntimeReplaceable - with ExpectsInputTypes { + with ExpectsInputTypes with TimeExpression { override def replacement: Expression = StaticInvoke( classOf[DateTimeUtils.type], @@ -316,7 +327,7 @@ object HourExpressionBuilder extends ExpressionBuilder { case class SecondsOfTimeWithFraction(child: Expression) extends RuntimeReplaceable - with ExpectsInputTypes { + with ExpectsInputTypes with TimeExpression { override def replacement: Expression = { val precision = child.dataType match { case TimeType(p) => p @@ -342,7 +353,7 @@ case class SecondsOfTimeWithFraction(child: Expression) case class SecondsOfTime(child: Expression) extends RuntimeReplaceable - with ExpectsInputTypes { + with ExpectsInputTypes with TimeExpression { override def replacement: Expression = StaticInvoke( classOf[DateTimeUtils.type], @@ -433,7 +444,8 @@ object SecondExpressionBuilder extends ExpressionBuilder { case class CurrentTime( child: Expression = Literal(TimeType.MICROS_PRECISION), timeZoneId: Option[String] = None) extends UnaryExpression - with TimeZoneAwareExpression with ImplicitCastInputTypes with CodegenFallback { + with TimeZoneAwareExpression with ImplicitCastInputTypes with CodegenFallback + with TimeExpression { def this() = { this(Literal(TimeType.MICROS_PRECISION), None) @@ -545,7 +557,7 @@ case class MakeTime( secsAndMicros: Expression) extends RuntimeReplaceable with ImplicitCastInputTypes - with ExpectsInputTypes { + with ExpectsInputTypes with TimeExpression { // Accept `sec` as DecimalType to avoid loosing precision of microseconds while converting // it to the fractional part of `sec`. If `sec` is an IntegerType, it can be cast into decimal @@ -570,7 +582,8 @@ case class MakeTime( * Adds day-time interval to time. */ case class TimeAddInterval(time: Expression, interval: Expression) - extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes { + extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes + with TimeExpression { override def nullIntolerant: Boolean = true override def left: Expression = time @@ -611,7 +624,8 @@ case class TimeAddInterval(time: Expression, interval: Expression) * Returns a day-time interval between time values. */ case class SubtractTimes(left: Expression, right: Expression) - extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes { + extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes + with TimeExpression { override def nullIntolerant: Boolean = true override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType, AnyTimeType) @@ -668,7 +682,8 @@ case class TimeDiff( end: Expression) extends TernaryExpression with RuntimeReplaceable - with ImplicitCastInputTypes { + with ImplicitCastInputTypes + with TimeExpression { override def first: Expression = unit override def second: Expression = start @@ -723,7 +738,8 @@ case class TimeDiff( since = "4.1.0") // scalastyle:on line.size.limit case class TimeTrunc(unit: Expression, time: Expression) - extends BinaryExpression with RuntimeReplaceable with ImplicitCastInputTypes { + extends BinaryExpression with RuntimeReplaceable with ImplicitCastInputTypes + with TimeExpression { override def left: Expression = unit override def right: Expression = time diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 77b775f6c49f7..1cee321846a46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4492,4 +4492,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "colType" -> "metadata", "errors" -> errors.mkString("- ", "\n- ", ""))) } + + def unsupportedTimeTypeError(): Throwable = { + new AnalysisException( + errorClass = "UNSUPPORTED_TIME_TYPE", + messageParameters = Map.empty) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 55c5eab5dda0b..36febe67f4096 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6621,6 +6621,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val TIME_TYPE_ENABLED = + buildConf("spark.sql.timeType.enabled") + .doc("When true, the TIME data type is supported.") + .version("4.1.0") + .booleanConf + .createWithDefault(Utils.isTesting) + /** * Holds information about keys that have been deprecated. * @@ -7781,6 +7788,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def coerceMergeNestedTypes: Boolean = getConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED) + def isTimeTypeEnabled: Boolean = getConf(SQLConf.TIME_TYPE_ENABLED) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index dc20af8e3700d..e3c392cd2a2c7 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -37,10 +37,11 @@ import org.apache.spark.sql.connect.config.Connect.{CONNECT_GRPC_ARROW_MAX_BATCH import org.apache.spark.sql.connect.planner.{InvalidInputErrors, SparkConnectPlanner} import org.apache.spark.sql.connect.service.ExecuteHolder import org.apache.spark.sql.connect.utils.MetricGenerator +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.{DoNotCleanup, LocalTableScanExec, QueryExecution, RemoveShuffleFiles, SkipMigration, SQLExecution} import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DataType, StructType, TimeType} import org.apache.spark.util.ThreadUtils /** @@ -133,6 +134,10 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", messageParameters = scala.collection.immutable.Map.empty) } + val timeTypeEnabled = spark.sessionState.conf.isTimeTypeEnabled + if (!timeTypeEnabled && schema.existsRecursively(_.isInstanceOf[TimeType])) { + throw QueryCompilationErrors.unsupportedTimeTypeError() + } val maxRecordsPerBatch = spark.sessionState.conf.arrowMaxRecordsPerBatch val timeZoneId = spark.sessionState.conf.sessionLocalTimeZone val largeVarTypes = spark.sessionState.conf.arrowUseLargeVarTypes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala index d43c9eab0a5ba..fb5b605bab01e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala @@ -93,6 +93,9 @@ object DataSourceUtils extends PredicateHelper { * in a driver side. */ def verifySchema(format: FileFormat, schema: StructType, readOnly: Boolean = false): Unit = { + if (!SQLConf.get.isTimeTypeEnabled && schema.existsRecursively(_.isInstanceOf[TimeType])) { + throw QueryCompilationErrors.unsupportedTimeTypeError() + } schema.foreach { field => val supported = if (readOnly) { format.supportReadDataType(field.dataType) diff --git a/sql/gen-sql-functions-docs.py b/sql/gen-sql-functions-docs.py index a1facbaaf7e3b..b49124ece0866 100644 --- a/sql/gen-sql-functions-docs.py +++ b/sql/gen-sql-functions-docs.py @@ -240,6 +240,8 @@ def generate_functions_examples_html(jvm, jspark, html_output_dir):

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b GIT binary patch literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q literal 0 HcmV?d00001 From ec3487883d4a488cf6e275b72299f4b26bf81fe0 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Mon, 24 Nov 2025 01:09:55 +0000 Subject: [PATCH 209/400] Preparing development version 4.1.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 4393175430265..0b0b67f74f48b 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.0 +Version: 4.1.1 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 743f74753b1f8..0bd7673d60185 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 1afbf81679906..f26b72b08efec 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 167582598a920..5a4dc11f51fcc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 8cede6cbf288e..794b8e9c6872e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 5e6348a717735..ece4683ea2a6c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 161802d43ca89..8e51d311be5f4 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 5e851791a6975..cc36b54df8519 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 869931e403349..130153d6127a7 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index dbac5e4552df5..114086da25cb3 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 1805f8462c278..981d484451865 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index adecb4b4185d3..4ff99df6350fc 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index bad1f7529c054..f6aa500d5f007 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index b5af1a093be36..dc01425a617fe 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 0022adff1a208..4f1eb8302aebc 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index b482aeb77a96d..f6b7dbcdc3c2a 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 5da0ee078e26e..758ea445746df 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index 2479590341822..cee71d4ccf453 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 9acc9b65d7653..57694e27cecaa 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index 54d5747e78aef..91f278500d78f 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 9d486a868c3de..356934c9eb929 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index a78b722eb6bf0..7abbc76899879 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 7ee7e096d38c8..545017f1d77d1 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2281f3e197f88..2719881e25cf4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index dc3890e4522ee..bda945e03c203 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.0 -SPARK_VERSION_SHORT: 4.1.0 +SPARK_VERSION: 4.1.1-SNAPSHOT +SPARK_VERSION_SHORT: 4.1.1 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.0"] + 'facetFilters': ["version:4.1.1"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index cd17565a1d98b..38daf00501ce6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 602817ca792e4..cc4327ccff090 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index fc593ff2c6196..66b8cb104e07f 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e49a0678d7182..aa8812cd9a1f6 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 6695e77228935..34df7e4b5be1d 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index b31c2b7fb3a24..9061edd455ebf 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 1824a28614bd3..1e61d457e20e9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 9b0e7c1f5ce40..3080cb455888b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.0" +__version__: str = "4.1.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 53bc41d397d60..74199753f2c5e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index c5f2749ea9fb0..3e58d34fc1857 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 3b471a8ff4133..6051496bf7fe6 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 6d0d0305ef710..f1a8c05034f60 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index e7997cf61a457..7d33b1e5f107e 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 8eef00489669d..d7b0d7e0554bb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index c6a5805bb0f5c..b2c5d4fa6fcb6 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 6e99700c90c51..0262d1af9fdb4 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 3b74dca497387..ecd735cb81ca5 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index ab19897f64285..24d74b505521f 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index febf7a2beddf4..79846203864e1 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e3aa8f1f3bb4c..270d23061f391 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 8c4087372913a..a1966cace77c8 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index e51872a36ca5b..55086c6b5de8d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 6e4a4d64f7930..94de01aede9bc 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index 9602aa01e7c4d..ffac010cad1cc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index cfc6a7150d858..934ff72708094 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml From 7fe9f5e23b9848cf056294341e443ee3116e4c46 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 24 Nov 2025 22:02:52 +0800 Subject: [PATCH 210/400] [SPARK-54115][TESTS][FOLLOW-UP] Refine `org.apache.spark.util.UtilsSuite` ### What changes were proposed in this pull request? ### Why are the changes needed? > Use doReturn() in those rare occasions when you cannot use when(Object) `when` is preferred in the official doc https://javadoc.io/static/org.mockito/mockito-core/5.12.0/org/mockito/Mockito.html#when(T) ### Does this PR introduce _any_ user-facing change? no, test-only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53192 from zhengruifeng/spark_54115_test_followup. Authored-by: Ruifeng Zheng Signed-off-by: Wenchen Fan (cherry picked from commit d14209c6ffba991b1d8ca9580708dea1ee37920e) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 51 +++++++++---------- 2 files changed, 25 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fc735ae3b99ea..721719f2a9767 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2089,7 +2089,7 @@ private[spark] object Utils val CONNECT_EXECUTE_THREAD_PREFIX = "SparkConnectExecuteThread" - private val threadInfoOrdering = Ordering.fromLessThan { + private[spark] val threadInfoOrdering = Ordering.fromLessThan { (threadTrace1: ThreadInfo, threadTrace2: ThreadInfo) => { def priority(ti: ThreadInfo): Int = ti.getThreadName match { case name if name.startsWith(TASK_THREAD_NAME_PREFIX) => 100 diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index d600260e9df2a..61952c4018534 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -38,8 +38,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext import org.apache.hadoop.ipc.{CallerContext => HadoopCallerContext} import org.apache.logging.log4j.Level -import org.mockito.Mockito.doReturn -import org.scalatest.PrivateMethodTester +import org.mockito.Mockito.when import org.scalatestplus.mockito.MockitoSugar.mock import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TaskContext} @@ -51,7 +50,7 @@ import org.apache.spark.scheduler.SparkListener import org.apache.spark.util.collection.Utils.createArray import org.apache.spark.util.io.ChunkedByteBufferInputStream -class UtilsSuite extends SparkFunSuite with ResetSystemProperties with PrivateMethodTester { +class UtilsSuite extends SparkFunSuite with ResetSystemProperties { test("timeConversion") { // Test -1 @@ -1132,37 +1131,35 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with PrivateMe test("ThreadInfoOrdering") { val task1T = mock[ThreadInfo] - doReturn(11L).when(task1T).getThreadId - doReturn("Executor task launch worker for task 1.0 in stage 1.0 (TID 11)") - .when(task1T).getThreadName - doReturn("Executor task launch worker for task 1.0 in stage 1.0 (TID 11)") - .when(task1T).toString + when(task1T.getThreadId).thenReturn(11L) + when(task1T.getThreadName) + .thenReturn("Executor task launch worker for task 1.0 in stage 1.0 (TID 11)") + when(task1T.toString) + .thenReturn("Executor task launch worker for task 1.0 in stage 1.0 (TID 11)") val task2T = mock[ThreadInfo] - doReturn(12L).when(task2T).getThreadId - doReturn("Executor task launch worker for task 2.0 in stage 1.0 (TID 22)") - .when(task2T).getThreadName - doReturn("Executor task launch worker for task 2.0 in stage 1.0 (TID 22)") - .when(task2T).toString + when(task2T.getThreadId).thenReturn(12L) + when(task2T.getThreadName) + .thenReturn("Executor task launch worker for task 2.0 in stage 1.0 (TID 22)") + when(task2T.toString) + .thenReturn("Executor task launch worker for task 2.0 in stage 1.0 (TID 22)") val connectExecuteOp1T = mock[ThreadInfo] - doReturn(21L).when(connectExecuteOp1T).getThreadId - doReturn("SparkConnectExecuteThread_opId=16148fb4-4189-43c3-b8d4-8b3b6ddd41c7") - .when(connectExecuteOp1T).getThreadName - doReturn("SparkConnectExecuteThread_opId=16148fb4-4189-43c3-b8d4-8b3b6ddd41c7") - .when(connectExecuteOp1T).toString + when(connectExecuteOp1T.getThreadId).thenReturn(21L) + when(connectExecuteOp1T.getThreadName) + .thenReturn("SparkConnectExecuteThread_opId=16148fb4-4189-43c3-b8d4-8b3b6ddd41c7") + when(connectExecuteOp1T.toString) + .thenReturn("SparkConnectExecuteThread_opId=16148fb4-4189-43c3-b8d4-8b3b6ddd41c7") val connectExecuteOp2T = mock[ThreadInfo] - doReturn(22L).when(connectExecuteOp2T).getThreadId - doReturn("SparkConnectExecuteThread_opId=4e4d1cac-ffde-46c1-b7c2-808b726cb47e") - .when(connectExecuteOp2T).getThreadName - doReturn("SparkConnectExecuteThread_opId=4e4d1cac-ffde-46c1-b7c2-808b726cb47e") - .when(connectExecuteOp2T).toString - - val threadInfoOrderingMethod = - PrivateMethod[Ordering[ThreadInfo]](Symbol("threadInfoOrdering")) + when(connectExecuteOp2T.getThreadId).thenReturn(22L) + when(connectExecuteOp2T.getThreadName) + .thenReturn("SparkConnectExecuteThread_opId=4e4d1cac-ffde-46c1-b7c2-808b726cb47e") + when(connectExecuteOp2T.toString) + .thenReturn("SparkConnectExecuteThread_opId=4e4d1cac-ffde-46c1-b7c2-808b726cb47e") + val sorted = Seq(connectExecuteOp1T, connectExecuteOp2T, task1T, task2T) - .sorted(Utils.invokePrivate(threadInfoOrderingMethod())) + .sorted(Utils.threadInfoOrdering) assert(sorted === Seq(task1T, task2T, connectExecuteOp1T, connectExecuteOp2T)) } From 981387066790b06e540e788cf1d9162098637eba Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 24 Nov 2025 22:30:59 +0800 Subject: [PATCH 211/400] [SPARK-51765][DOCS][4.1] Document SQL Scripting Syntax Porting https://github.com/apache/spark/pull/53155 to `branch-4.1` and resolving merge conflicts. The only difference is a minor note in `docs/sql-ref-scripting.md` on how to enable SQL Scripting, since it's not enabled by default in Spark 4.1. ### What changes were proposed in this pull request? Adding documentation for SQL Scripting feature. ### Why are the changes needed? If it is not documented, it never happened :-) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Documentation change, reviewed manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53194 from davidm-db/scripting-docs-4.1. Lead-authored-by: Serge Rielau Co-authored-by: David Milicevic Co-authored-by: David Milicevic <163021185+davidm-db@users.noreply.github.com> Signed-off-by: Wenchen Fan --- docs/_data/menu-sql.yaml | 6 + docs/control-flow/case-stmt.md | 102 +++++++ docs/control-flow/compound-stmt.md | 164 ++++++++++ docs/control-flow/for-stmt.md | 92 ++++++ docs/control-flow/if-stmt.md | 71 +++++ docs/control-flow/iterate-stmt.md | 70 +++++ docs/control-flow/leave-stmt.md | 71 +++++ docs/control-flow/loop-stmt.md | 83 +++++ docs/control-flow/repeat-stmt.md | 84 +++++ docs/control-flow/while-stmt.md | 83 +++++ docs/sql-ref-name-resolution.md | 423 ++++++++++++++++++++++++++ docs/sql-ref-scripting.md | 89 ++++++ docs/sql-ref-syntax-qry-select-tvf.md | 2 +- docs/sql-ref-syntax.md | 14 + docs/sql-ref.md | 3 + 15 files changed, 1356 insertions(+), 1 deletion(-) create mode 100644 docs/control-flow/case-stmt.md create mode 100644 docs/control-flow/compound-stmt.md create mode 100644 docs/control-flow/for-stmt.md create mode 100644 docs/control-flow/if-stmt.md create mode 100644 docs/control-flow/iterate-stmt.md create mode 100644 docs/control-flow/leave-stmt.md create mode 100644 docs/control-flow/loop-stmt.md create mode 100644 docs/control-flow/repeat-stmt.md create mode 100644 docs/control-flow/while-stmt.md create mode 100644 docs/sql-ref-name-resolution.md create mode 100644 docs/sql-ref-scripting.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index b1688aec57f01..f530d78dae453 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -99,6 +99,10 @@ url: sql-ref-literals.html - text: Null Semantics url: sql-ref-null-semantics.html + - text: Name Resolution + url: sql-ref-name-resolution.html + - text: SQL Scripting + url: sql-ref-scripting.html - text: SQL Syntax url: sql-ref-syntax.html subitems: @@ -108,6 +112,8 @@ url: sql-ref-syntax.html#dml-statements - text: Data Retrieval(Queries) url: sql-ref-syntax.html#data-retrieval-statements + - text: SQL Scripting Statements + url: sql-ref-syntax.html#sql-scripting-statements - text: Auxiliary Statements url: sql-ref-syntax.html#auxiliary-statements - text: Pipe Syntax diff --git a/docs/control-flow/case-stmt.md b/docs/control-flow/case-stmt.md new file mode 100644 index 0000000000000..c92663905b061 --- /dev/null +++ b/docs/control-flow/case-stmt.md @@ -0,0 +1,102 @@ +--- +layout: global +title: CASE statement +displayTitle: CASE statement +license: | + 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. +--- + +Executes `thenStmtN` for the first `optN` that equals `expr` or `elseStmt` if no `optN` matches `expr`. +This is called a _simple case statement_. + +Executes `thenStmtN` for the first `condN` evaluating to `true`, or `elseStmt` if no `condN` evaluates to `true`. +This is called a _searched case statement_. + +For case expressions that yield result values, see `CASE expression`) + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +CASE expr + { WHEN opt THEN { thenStmt ; } [...] } [...] + [ ELSE { elseStmt ; } [...] ] +END CASE + +CASE + { WHEN cond THEN { thenStmt ; } [...] } [...] + [ ELSE { elseStmt ; } [...] ] +END CASE +``` + +## Parameters + +- **`expr`**: Any expression for which a comparison is defined. +- **`opt`**: An expression with a least common type with `expr` and all other `optN`. +- **`thenStmt`**: A SQL Statement to execute if preceding condition is `true`. +- **`elseStmt`**: A SQL Statement to execute if no condition is `true`. +- **`cond`**: A `BOOLEAN` expression. + +Conditions are evaluated in order, and only the first set of `stmt` for which `opt` or `cond` evaluate to true will be executed. + +## Examples + +```SQL +-- a simple case statement +> BEGIN + DECLARE choice INT DEFAULT 3; + DECLARE result STRING; + CASE choice + WHEN 1 THEN + VALUES ('one fish'); + WHEN 2 THEN + VALUES ('two fish'); + WHEN 3 THEN + VALUES ('red fish'); + WHEN 4 THEN + VALUES ('blue fish'); + ELSE + VALUES ('no fish'); + END CASE; + END; + red fish + +-- A searched case statement +> BEGIN + DECLARE choice DOUBLE DEFAULT 3.9; + DECLARE result STRING; + CASE + WHEN choice < 2 THEN + VALUES ('one fish'); + WHEN choice < 3 THEN + VALUES ('two fish'); + WHEN choice < 4 THEN + VALUES ('red fish'); + WHEN choice < 5 OR choice IS NULL THEN + VALUES ('blue fish'); + ELSE + VALUES ('no fish'); + END CASE; + END; + red fish +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [compound statement](compound-stmt.html) +- [IF statement](if-stmt.html) diff --git a/docs/control-flow/compound-stmt.md b/docs/control-flow/compound-stmt.md new file mode 100644 index 0000000000000..d34e70648de43 --- /dev/null +++ b/docs/control-flow/compound-stmt.md @@ -0,0 +1,164 @@ +--- +layout: global +title: compound statement +displayTitle: compound statement +license: | + 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. +--- + +Implements a SQL Script block that can contain a sequence of SQL statements, control-of-flow statements, local variable declarations, and exception handlers. + +## Syntax + +``` +[ label : ] + BEGIN + [ { declare_variable | declare_condition } ; [...] ] + [ declare_handler ; [...] ] + [ SQL_statement ; [...] ] + END [ label ] + +declare_variable + DECLARE variable_name [, ...] datatype [ DEFAULT default_expr ] + +declare_condition + DECLARE condition_name CONDITION [ FOR SQLSTATE [ VALUE ] sqlstate ] + +declare_handler + DECLARE handler_type HANDLER FOR condition_values handler_action + +handler_type + EXIT + +condition_values + { { SQLSTATE [ VALUE ] sqlstate | condition_name } [, ...] | + { SQLEXCEPTION | NOT FOUND } [, ...] } +``` + +## Parameters + +- **`label`** + + An optional identifier is used to qualify variables defined within the compound and to leave the compound. + Both label occurrences must match, and the `END` label can only be specified if `label:` is specified. + + `label` must not be specified for a top level compound statement. + +- **`NOT ATOMIC`** + + Specifies that, if an SQL statement within the compound fails, previous SQL statements will not be rolled back. + This is the default and only behavior. + +- **`declare_variable`** + + A local variable declaration for one or more variables + + - **`variable_name`** + + A name for the variable. + The name must not be qualified, and be unique within the compound statement. + + - **`data_type`** + + Any supported data type. If data_type is omitted, you must specify DEFAULT, and the type is derived from the default_expression. + + - **`{ DEFAULT | = } default_expression`** + + Defines the variable's initial value after declaration. default_expression must be castable to data_type. If no default is specified, the variable is initialized with NULL. + +- **`Declare_condition`** + + A local condition declaration + + - **`condition_name`** + + The unqualified name of the condition is scoped to the compound statement. + + - **`sqlstate`** + + A `STRING` literal of 5 alphanumeric characters (case insensitive) consisting of A-Z and 0..9. The SQLSTATE must not start with ‘00’, ‘01’, or ‘XX’. Any SQLSTATE starting with ‘02’ will be caught by the predefined NOT FOUND exception as well. If not specified, the SQLSTATE is ‘45000’. + +- **`declare_handler`** + + A declaration for an error handler. + + - **`handler_type`** + + - **`EXIT`** + + Classifies the handler to exit the compound statement after the condition is handled. + + - **`condition_values`** + + Specifies to which sqlstates or conditions the handler applies. + Condition values must be unique within all handlers within the compound statement. + Specific condition values take precedence over `SQLEXCEPTION`. + + - **`sqlstate`** + + A `STRING` literal of 5 characters `'A'-'Z'` and `'0'-'9'` (case insensitive). + + - **`condition_name`** + + A condition defined within this compound, an outer compound statement, or a system-defined error class. + + - **`SQLEXCEPTION`** + + Applies to any user-facing error condition. + + - **`NOT FOUND`** + + Applies to any error condition with a SQLSTATE ‘02’ class. + + - **`handler_action`** + + A SQL statement to execute when any of the condition values occur. + To add multiple statements, use a nested compound statement. + +- **`SQL_statement`** + + A SQL statement such as a DDL, DML, control statement, or compound statement. + Any `SELECT` or `VALUES` statement produces a result set that the invoker can consume. + +## Examples + +```SQL +-- A compound statement with local variables, and exit hanlder and a nested compound. +> BEGIN + DECLARE a INT DEFAULT 1; + DECLARE b INT DEFAULT 5; + DECLARE EXIT HANDLER FOR DIVIDE_BY_ZERO + div0: BEGIN + VALUES (15); + END div0; + SET a = 10; + SET a = b / 0; + VALUES (a); +END; +15 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [IF Statement](../control-flow/if-stmt.html) +- [LOOP Statement](../control-flow/loop-stmt.html) +- [WHILE Statement](../control-flow/while-stmt.html) +- [REPEAT Statement](../control-flow/repeat-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) +- [ITERATE Statement](../control-flow/iterate-stmt.html) +- [LEAVE Statement](../control-flow/leave-stmt.html) diff --git a/docs/control-flow/for-stmt.md b/docs/control-flow/for-stmt.md new file mode 100644 index 0000000000000..25a1cfa7218ec --- /dev/null +++ b/docs/control-flow/for-stmt.md @@ -0,0 +1,92 @@ +--- +layout: global +title: FOR statement +displayTitle: FOR statement +license: | + 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. +--- + +Repeat the execution of a list of statements for each row returned by query. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +[ label : ] FOR [ variable_name AS ] query + DO + { stmt ; } [...] + END FOR [ label ] +``` + +## Parameters + +- **label** + + An optional label for the loop which is unique amongst all labels for statements within which the `FOR` statement is contained. + If an end label is specified, it must match the beginning label. + The label can be used to [LEAVE](leave-stmt.html) or [ITERATE](iterate-stmt.html) the loop. + To qualify loop column references, use the `variable_name`, not the `label`. + +- **variable_name** + + An optional name you can use as a qualifier when referencing the columns in the cursor. + +- **stmt** + + A SQL statement + +## Notes + +If the query operates on a table that is also modified within the loop's body, the semantics depend on the data source. +For Delta tables, the query will remain unaffected. +Spark does not guarantee the full execution of the query if the `FOR` loop completes prematurely due to a `LEAVE` statement or an error condition. +When exceptions or side-effects occur during the execution of the query, Spark does not guarantee at which point in time within the loop these occur. +Often `FOR` loops can be replaced with relational queries, which are typically more efficient. + +## Examples + +```SQL +-- sum up all odd numbers from 1 through 10 +> BEGIN + DECLARE sum INT DEFAULT 0; + sumNumbers: FOR row AS SELECT num FROM range(1, 20) AS t(num) DO + IF num > 10 THEN + LEAVE sumNumbers; + ELSEIF num % 2 = 0 THEN + ITERATE sumNumbers; + END IF; + SET sum = sum + row.num; + END FOR sumNumbers; + VALUES (sum); + END; + 25 + +-- Compare with the much more efficient relational computation: +> SELECT sum(num) FROM range(1, 10) AS t(num) WHERE num % 2 = 1; + 25 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [LOOP Statement](../control-flow/loop-stmt.html) +- [WHILE Statement](../control-flow/while-stmt.html) +- [REPEAT Statement](../control-flow/repeat-stmt.html) +- [LEAVE Statement](../control-flow/leave-stmt.html) +- [ITERATE Statement](../control-flow/iterate-stmt.html) diff --git a/docs/control-flow/if-stmt.md b/docs/control-flow/if-stmt.md new file mode 100644 index 0000000000000..2e93dc7f6d505 --- /dev/null +++ b/docs/control-flow/if-stmt.md @@ -0,0 +1,71 @@ +--- +layout: global +title: IF statement +displayTitle: IF statement +license: | + 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. +--- + +Executes lists of statements based on the first condition that evaluates to true. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +IF condition THEN { stmt ; } [...] + [ { ELSEIF condition THEN { stmt ; } [...] } [...] ] + [ ELSE { stmt ; } [...] ] + END IF +``` + +## Parameters + +- **condition** + + Any expression evaluating to a BOOLEAN. + +- **stmt** + + A SQL statement to execute if the `condition` is `true`. + +## Examples + +```SQL +> BEGIN + DECLARE choice DOUBLE DEFAULT 3.9; + DECLARE result STRING; + IF choice < 2 THEN + VALUES ('one fish'); + ELSEIF choice < 3 THEN + VALUES ('two fish'); + ELSEIF choice < 4 THEN + VALUES ('red fish'); + ELSEIF choice < 5 OR choice IS NULL THEN + VALUES ('blue fish'); + ELSE + VALUES ('no fish'); + END IF; + END; + red fish +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) diff --git a/docs/control-flow/iterate-stmt.md b/docs/control-flow/iterate-stmt.md new file mode 100644 index 0000000000000..d73f33a26bf95 --- /dev/null +++ b/docs/control-flow/iterate-stmt.md @@ -0,0 +1,70 @@ +--- +layout: global +title: ITERATE statement +displayTitle: ITERATE statement +license: | + 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. +--- + +Terminates the execution of an iteration of a looping statement and continues with the next iteration if the looping condition is met. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +ITERATE label +``` + +## Parameters + +- **label** + + The label identifies a looping statement that contains the `ITERATE` statement directly or indirectly. + +## Examples + +```SQL +-- sum up all odd numbers from 1 through 10 +> BEGIN + DECLARE sum INT DEFAULT 0; + DECLARE num INT DEFAULT 0; + sumNumbers: LOOP + SET num = num + 1; + IF num > 10 THEN + LEAVE sumNumbers; + END IF; + IF num % 2 = 0 THEN + ITERATE sumNumbers; + END IF; + SET sum = sum + num; + END LOOP sumNumbers; + VALUES (sum); + END; +25 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) +- [LOOP Statement](../control-flow/loop-stmt.html) +- [WHILE Statement](../control-flow/while-stmt.html) +- [REPEAT Statement](../control-flow/repeat-stmt.html) +- [IF Statement](../control-flow/if-stmt.html) +- [LEAVE Statement](../control-flow/leave-stmt.html) diff --git a/docs/control-flow/leave-stmt.md b/docs/control-flow/leave-stmt.md new file mode 100644 index 0000000000000..a705c48b239a3 --- /dev/null +++ b/docs/control-flow/leave-stmt.md @@ -0,0 +1,71 @@ +--- +layout: global +title: LEAVE statement +displayTitle: LEAVE statement +license: | + 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. +--- + +Terminates the execution of an iteration of a looping statement and continues with the next iteration if the looping condition is met. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +ITERATE label +``` + +## Parameters + +- **label** + + The label identifies a statement to leave that directly or indirectly contains the `LEAVE` statement. + +## Examples + +```SQL +-- sum up all odd numbers from 1 through 10 +-- Iterate over even numbers and leave the loop after 10 has been reached. +> BEGIN + DECLARE sum INT DEFAULT 0; + DECLARE num INT DEFAULT 0; + sumNumbers: LOOP + SET num = num + 1; + IF num > 10 THEN + LEAVE sumNumbers; + END IF; + IF num % 2 = 0 THEN + ITERATE sumNumbers; + END IF; + SET sum = sum + num; + END LOOP sumNumbers; + VALUES (sum); + END; +25 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) +- [LOOP Statement](../control-flow/loop-stmt.html) +- [WHILE Statement](../control-flow/while-stmt.html) +- [IF Statement](../control-flow/if-stmt.html) +- [ITERATE Statement](../control-flow/iterate-stmt.html) + diff --git a/docs/control-flow/loop-stmt.md b/docs/control-flow/loop-stmt.md new file mode 100644 index 0000000000000..7ca3b3b5bbf96 --- /dev/null +++ b/docs/control-flow/loop-stmt.md @@ -0,0 +1,83 @@ +--- +layout: global +title: LOOP statement +displayTitle: LOOP statement +license: | + 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. +--- + +Repeat the execution of a list of statements. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +[ label : ] LOOP + { stmt ; } [...] + END LOOP [ label ] +``` + +## Parameters + +- **label** + + An optional label for the loop, which is unique amongst all labels for statements within which the `LOOP` statement is contained. + If an end label is specified, it must match the beginning label. + The label can be used to [LEAVE](leave-stmt.html) or [ITERATE](iterate-stmt.html) the loop. + +- **stmt** + + A SQL statement + +## Examples + +```SQL +-- sum up all odd numbers from 1 through 10 +> BEGIN + DECLARE sum INT DEFAULT 0; + DECLARE num INT DEFAULT 0; + sumNumbers: LOOP + SET num = num + 1; + IF num > 10 THEN + LEAVE sumNumbers; + END IF; + IF num % 2 = 0 THEN + ITERATE sumNumbers; + END IF; + SET sum = sum + num; + END LOOP sumNumbers; + VALUES (sum); + END; + 25 + +-- Compare with the much more efficient relational computation: +> SELECT sum(num) FROM range(1, 10) AS t(num) WHERE num % 2 = 1; + 25 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) +- [WHILE Statement](../control-flow/while-stmt.html) +- [REPEAT Statement](../control-flow/repeat-stmt.html) +- [IF Statement](../control-flow/if-stmt.html) +- [ITERATE Statement](../control-flow/iterate-stmt.html) +- [LEAVE Statement](../control-flow/leave-stmt.html) + diff --git a/docs/control-flow/repeat-stmt.md b/docs/control-flow/repeat-stmt.md new file mode 100644 index 0000000000000..4d28a6b05e0e4 --- /dev/null +++ b/docs/control-flow/repeat-stmt.md @@ -0,0 +1,84 @@ +--- +layout: global +title: REPEAT statement +displayTitle: REPEAT statement +license: | + 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. +--- + +Repeat the execution of a list of statements until a condition is true. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +[ label : ] REPEAT + { stmt ; } [...] + UNTIL cond + END REPEAT [ label ] +``` + +## Parameters + +- **label** + + An optional label for the loop, which is unique amongst all labels for statements within which the `REPEAT` statement is contained. + The label can be used to [LEAVE](leave-stmt.html) or [ITERATE](iterate-stmt.html) the loop. + +- **cond** + + Any expression evaluating to a BOOLEAN + +- **stmt** + + A SQL statement + +## Examples + +```SQL +-- sum up all odd numbers from 1 through 10 +> BEGIN + DECLARE sum INT DEFAULT 0; + DECLARE num INT DEFAULT 0; + sumNumbers: REPEAT + SET num = num + 1; + IF num % 2 = 0 THEN + ITERATE sumNumbers; + END IF; + SET sum = sum + num; + UNTIL num = 10 + END REPEAT sumNumbers; + VALUES (sum); + END; + 25 + +-- Compare with the much more efficient relational computation: +> SELECT sum(num) FROM range(1, 10) AS t(num) WHERE num % 2 = 1; + 25 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) +- [IF Statement](../control-flow/if-stmt.html) +- [ITERATE Statement](../control-flow/iterate-stmt.html) +- [WHILE Statement](../control-flow/while-stmt.html) +- [LEAVE Statement](../control-flow/leave-stmt.html) +- [LOOP Statement](../control-flow/loop-stmt.html) diff --git a/docs/control-flow/while-stmt.md b/docs/control-flow/while-stmt.md new file mode 100644 index 0000000000000..0edf77f0cba0d --- /dev/null +++ b/docs/control-flow/while-stmt.md @@ -0,0 +1,83 @@ +--- +layout: global +title: WHILE statement +displayTitle: WHILE statement +license: | + 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. +--- + +Repeat the execution of a list of statements while a condition is true. + +This statement may only be used within a [compound statement](compound-stmt.html). + +## Syntax + +``` +[ label : ] WHILE cond DO + { stmt ; } [...] + END WHILE [ label ] +``` + +## Parameters + +- **label** + + An optional label for the loop, which is unique amongst all labels for statements within which the `WHILE` statement is contained. + The label can be used to [LEAVE](leave-stmt.html) or [ITERATE](iterate-stmt.html) the loop. + +- **cond** + + Any expression evaluating to a `BOOLEAN`. + +- **stmt** + + A SQL statement. + +## Examples + +```SQL +-- sum up all odd numbers from 1 through 10 +> BEGIN + DECLARE sum INT DEFAULT 0; + DECLARE num INT DEFAULT 0; + sumNumbers: WHILE num < 10 DO + SET num = num + 1; + IF num % 2 = 0 THEN + ITERATE sumNumbers; + END IF; + SET sum = sum + num; + END WHILE sumNumbers; + VALUES (sum); + END; + 25 + +-- Compare with the much more efficient relational computation: +> SELECT sum(num) FROM range(1, 10) AS t(num) WHERE num % 2 = 1; + 25 +``` + +## Related articles + +- [SQL Scripting](../sql-ref-scripting.html) +- [CASE Statement](../control-flow/case-stmt.html) +- [Compound Statement](../control-flow/compound-stmt.html) +- [FOR Statement](../control-flow/for-stmt.html) +- [REPEAT Statement](../control-flow/repeat-stmt.html) +- [IF Statement](../control-flow/if-stmt.html) +- [ITERATE Statement](../control-flow/iterate-stmt.html) +- [LEAVE Statement](../control-flow/leave-stmt.html) +- [LOOP Statement](../control-flow/loop-stmt.html) + diff --git a/docs/sql-ref-name-resolution.md b/docs/sql-ref-name-resolution.md new file mode 100644 index 0000000000000..2532f05e164b3 --- /dev/null +++ b/docs/sql-ref-name-resolution.md @@ -0,0 +1,423 @@ +--- +layout: global +title: Name Resolution +displayTitle: Name Resolution +license: | + 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 resolution is the process by which [identifiers](sql-ref-identifier.html) are resolved to specific column-, field-, parameter-, or table-references. + +## Column, field, parameter, and variable resolution + +Identifiers in expressions can be references to any one of the following: + +- Column name based on a view, table, common table expression (CTE), or a column_alias. +- Field name or map key within a struct or map. + Fields and keys can never be unqualified. +- Parameter name of a SQL User Defined Function. +- Session or SQL script local variable name. +- A special function such as `current_user` or `current_date` which does not require the usage of `()`. +- The `DEFAULT` keyword which is used in the context of `INSERT`, or `SET VARIABLE` to set a column or variable value to its default. + +Name resolution applies the following principles: + +- The _closest_ matching reference wins, and +- Columns and parameter win over fields and keys. + +In detail, resolution of identifiers to a specific reference follows these rules in order: + +1. **Local references** + + 1. **Column reference** + + Match the identifier, which may be qualified, to a column name in a table reference of the `FROM clause`. + + If there is more than one such match, raise an AMBIGUOUS_COLUMN_OR_FIELD error. + + 1. **Parameterless function reference** + + If the identifier is unqualified and matches `current_user`, `current_date`, or `current_timestamp`: Resolve it as one of these functions. + + 1. **Column DEFAULT specification** + + If the identifier is unqualified, matches `default` and makes up the entire expression in the context of an `UPDATE SET`, `INSERT VALUES`, or `MERGE WHEN [NOT] MATCHED`: Resolve as the respective `DEFAULT` value of the target table of the `INSERT`. + + 1. **Struct field or map key reference** + + If the identifier is qualified, then endeavor to match it to a field or map key according to the following steps: + + A. Remove the last identifier and treat it as a field or key. + + B. Match the remainder to a column in table reference of the `FROM clause`. + + - If there is more than one such match, raise an AMBIGUOUS_COLUMN_OR_FIELD error. + + - If there is a match and the column is a: + + - **`STRUCT`**: Match the field. + + If the field cannot be matched, raise a FIELD_NOT_FOUND error. + + If there is more than one field, raise a AMBIGUOUS_COLUMN_OR_FIELD error. + + - **`MAP`**: Raise an error if the key is qualified. + + A runtime error may occur if the key is not actually present in the map. + + - Any other type: Raise an error. + + C. Repeat the preceding step to remove the trailing identifier as a field. Apply rules (A) and (B) while there is an identifier left to interpret as a column. + +1. **Lateral column aliasing** + + If the expression is within a `SELECT` list, match the leading identifier to a preceding column alias in that `SELECT` list. + + If there is more than one such match, raise an AMBIGUOUS_LATERAL_COLUMN_ALIAS error. + + Match each remaining identifier as a field or a map key, and raise FIELD_NOT_FOUND or AMBIGUOUS_COLUMN_OR_FIELD error if they cannot be matched. + +1. **Correlation** + + - **LATERAL** + + If the query is preceded by a `LATERAL` keyword, apply rules 1.a and 1.d considering the table references in the `FROM` containing the query and preceding the `LATERAL`. + + - **Regular** + + If the query is a scalar subquery, `IN`, or `EXISTS` subquery apply rules 1.a, 1.d, and 2 considering the table references in the containing query’s `FROM` clause. + +1. **Nested correlation** + + Re-apply rule 3 iterating over the nesting levels of the query. + +1. **[FOR loop](control-flow/for-stmt.md]** + + If the statement is contained in a `FOR` loop: + + A. Match the identifier to a column in a `FOR` loop statement query. + + If the identifier is qualified, the qualifier must match the name of the FOR loop variable if defined. + + B. If the identifier is qualified, match to a field or map key of a parameter following rule 1.c + +1. **[Compound statement](control-flow/compound-stmt.html)** + + If the statement is contained in a compound statement: + + A. Match the identifier to a variable declared in that compound statement. + + If the identifier is qualified, the qualifier must match the label of the compound statement if one was defined. + + B. If the identifier is qualified, match to a field or map key of a variable following rule 1.c + +1. **Nested compound statement or `FOR` loop** + + Re-apply rules 5 and 6, iterating over the nesting levels of the compound statement. + +1. **Routine parameters** + + If the expression is part of a CREATE FUNCTION statement: + + 1. Match the identifier to a parameter name. If the identifier is qualified, the qualifier must match the name of the routine. + 1. If the identifier is qualified, match to a field or map key of a parameter following rule 1.c + +1. **Session Variables** + + 1. Match the identifier to a variable name. If the identifier is qualified, the qualifier must be `session` or `system.session`. + 1. If the identifier is qualified, match to a field or map key of a variable following rule 1.c + +### Limitations + +To prevent execution of potentially expensive correlated queries, Spark limits supported correlation to one level. +This restriction also applies to parameter references in SQL functions. + +### Examples + +```sql +-- Differentiating columns and fields +> SELECT a FROM VALUES(1) AS t(a); + 1 + +> SELECT t.a FROM VALUES(1) AS t(a); + 1 + +> SELECT t.a FROM VALUES(named_struct('a', 1)) AS t(t); + 1 + +-- A column takes precendece over a field +> SELECT t.a FROM VALUES(named_struct('a', 1), 2) AS t(t, a); + 2 + +-- Implict lateral column alias +> SELECT c1 AS a, a + c1 FROM VALUES(2) AS T(c1); + 2 4 + +-- A local column reference takes precedence, over a lateral column alias +> SELECT c1 AS a, a + c1 FROM VALUES(2, 3) AS T(c1, a); + 2 5 + +-- A scalar subquery correlation to S.c3 +> SELECT (SELECT c1 FROM VALUES(1, 2) AS t(c1, c2) + WHERE t.c2 * 2 = c3) + FROM VALUES(4) AS s(c3); + 1 + +-- A local reference takes precedence over correlation +> SELECT (SELECT c1 FROM VALUES(1, 2, 2) AS t(c1, c2, c3) + WHERE t.c2 * 2 = c3) + FROM VALUES(4) AS s(c3); + NULL + +-- An explicit scalar subquery correlation to s.c3 +> SELECT (SELECT c1 FROM VALUES(1, 2, 2) AS t(c1, c2, c3) + WHERE t.c2 * 2 = s.c3) + FROM VALUES(4) AS s(c3); + 1 + +-- Correlation from an EXISTS predicate to t.c2 +> SELECT c1 FROM VALUES(1, 2) AS T(c1, c2) + WHERE EXISTS(SELECT 1 FROM VALUES(2) AS S(c2) + WHERE S.c2 = T.c2); + 1 + +-- Attempt a lateral correlation to t.c2 +> SELECT c1, c2, c3 + FROM VALUES(1, 2) AS t(c1, c2), + (SELECT c3 FROM VALUES(3, 4) AS s(c3, c4) + WHERE c4 = c2 * 2); + [UNRESOLVED_COLUMN] `c2` + +-- Successsful usage of lateral correlation with keyword LATERAL +> SELECT c1, c2, c3 + FROM VALUES(1, 2) AS t(c1, c2), + LATERAL(SELECT c3 FROM VALUES(3, 4) AS s(c3, c4) + WHERE c4 = c2 * 2); + 1 2 3 + +-- Referencing a parameter of a SQL function +> CREATE OR REPLACE TEMPORARY FUNCTION func(a INT) RETURNS INT + RETURN (SELECT c1 FROM VALUES(1) AS T(c1) WHERE c1 = a); +> SELECT func(1), func(2); + 1 NULL + +-- A column takes precedence over a parameter +> CREATE OR REPLACE TEMPORARY FUNCTION func(a INT) RETURNS INT + RETURN (SELECT a FROM VALUES(1) AS T(a) WHERE t.a = a); +> SELECT func(1), func(2); + 1 1 + +-- Qualify the parameter with the function name +> CREATE OR REPLACE TEMPORARY FUNCTION func(a INT) RETURNS INT + RETURN (SELECT a FROM VALUES(1) AS T(a) WHERE t.a = func.a); +> SELECT func(1), func(2); + 1 NULL + +-- Lateral alias takes precedence over correlated reference +> SELECT (SELECT c2 FROM (SELECT 1 AS c1, c1 AS c2) WHERE c2 > 5) + FROM VALUES(6) AS t(c1) + NULL + +-- Lateral alias takes precedence over function parameters +> CREATE OR REPLACE TEMPORARY FUNCTION func(x INT) + RETURNS TABLE (a INT, b INT, c DOUBLE) + RETURN SELECT x + 1 AS x, x +> SELECT * FROM func(1) + 2 2 + +-- All together now +> CREATE OR REPLACE TEMPORARY VIEW lat(a, b) AS VALUES('lat.a', 'lat.b'); + +> CREATE OR REPLACE TEMPORARY VIEW frm(a) AS VALUES('frm.a'); + +> CREATE OR REPLACE TEMPORARY FUNCTION func(a INT, b int, c int) + RETURNS TABLE + RETURN SELECT t.* + FROM lat, + LATERAL(SELECT a, b, c + FROM frm) AS t; + +> VALUES func('func.a', 'func.b', 'func.c'); + a b c + ----- ----- ------ + frm.a lat.b func.c +``` + +## Table and view resolution + +An identifier in table-reference can be any one of the following: + +- Persistent table or view +- Common table expression (CTE) +- [Temporary view](sql-ref-syntax-ddl-create-view.html) + +Resolution of an identifier depends on whether it is qualified: + +- **Qualified** + + If the identifier is fully qualified with three parts: `catalog.schema.relation`, it is unique. + + If the identifier consists of two parts: `schema.relation`, it is further qualified with the result of `SELECT current_catalog()` to make it unique. + +- **Unqualified** + + 1. **Common table expression** + + If the reference is within the scope of a `WITH` clause, match the identifier to a CTE starting with the immediately containing `WITH` clause and moving outwards from there. + + 1. **Temporary view** + + Match the identifier to any temporary view defined within the current session. + + 1. **Persisted table** + + Fully qualify the identifier by pre-pending the result of `SELECT current_catalog()` and `SELECT current_schema()` and look it up as a persistent relation. + +If the relation cannot be resolved to any table, view, or CTE, Databricks raises a TABLE_OR_VIEW_NOT_FOUND error. + +### Examples + +```sql +-- Setting up a scenario +> USE CATALOG spark_catalog; +> USE SCHEMA default; + +> CREATE TABLE rel(c1 int); +> INSERT INTO rel VALUES(1); + +-- An fully qualified reference to rel: +> SELECT c1 FROM spark_catalog.default.rel; + 1 + +-- A partially qualified reference to rel: +> SELECT c1 FROM default.rel; + 1 + +-- An unqualified reference to rel: +> SELECT c1 FROM rel; + 1 + +-- Add a temporary view with a conflicting name: +> CREATE TEMPORARY VIEW rel(c1) AS VALUES(2); + +-- For unqualified references the temporary view takes precedence over the persisted table: +> SELECT c1 FROM rel; + 2 + +-- Temporary views cannot be qualified, so qualifiecation resolved to the table: +> SELECT c1 FROM default.rel; + 1 + +-- An unqualified reference to a common table expression wins even over a temporary view: +> WITH rel(c1) AS (VALUES(3)) + SELECT * FROM rel; + 3 + +-- If CTEs are nested, the match nearest to the table reference takes precedence. +> WITH rel(c1) AS (VALUES(3)) + (WITH rel(c1) AS (VALUES(4)) + SELECT * FROM rel); + 4 + +-- To resolve the table instead of the CTE, qualify it: +> WITH rel(c1) AS (VALUES(3)) + (WITH rel(c1) AS (VALUES(4)) + SELECT * FROM default.rel); + 1 + +-- For a CTE to be visible it must contain the query +> SELECT * FROM (WITH cte(c1) AS (VALUES(1)) + SELECT 1), + cte; + [TABLE_OR_VIEW_NOT_FOUND] The table or view `cte` cannot be found. +``` + +## Function resolution + +A function reference is recognized by the mandatory trailing set of parentheses. + +It can resolve to: + +- A builtin function provided by Spark, +- A temporary user defined function scoped to the current session, or +- A persistent user defined function. + +Resolution of a function name depends on whether it is qualified: + +- **Qualified** + + If the name is fully qualified with three parts: `catalog.schema.function`, it is unique. + + If the name consists of two parts: `schema.function`, it is further qualified with the result of `SELECT current_catalog()` to make it unique. + + The function is then looked up in the catalog. + +- **Unqualified** + + For unqualified function names Spark follows a fixed order of precedence (`PATH`): + + 1. **Builtin function** + + If a function by this name exists among the set of built-in functions, that function is chosen. + + 1. **Temporary function** + + If a function by this name exists among the set of temporary functions, that function is chosen. + + 1. **Persisted function** + + Fully qualify the function name by pre-pending the result of `SELECT current_catalog()` and `SELECT current_schema()` and look it up as a persistent function. + +If the function cannot be resolved Spark raises an `UNRESOLVED_ROUTINE` error. + +### Examples + +```sql +> USE CATALOG spark_catalog; +> USE SCHEMA default; + +-- Create a function with the same name as a builtin +> CREATE FUNCTION concat(a STRING, b STRING) RETURNS STRING + RETURN b || a; + +-- unqualified reference resolves to the builtin CONCAT +> SELECT concat('hello', 'world'); + helloworld + +-- Qualified reference resolves to the persistent function +> SELECT default.concat('hello', 'world'); + worldhello + +-- Create a persistent function +> CREATE FUNCTION func(a INT, b INT) RETURNS INT + RETURN a + b; + +-- The persistent function is resolved without qualifying it +> SELECT func(4, 2); + 6 + +-- Create a conflicting temporary function +> CREATE FUNCTION func(a INT, b INT) RETURNS INT + RETURN a / b; + +-- The temporary function takes precedent +> SELECT func(4, 2); + 2 + +-- To resolve the persistent function it now needs qualification +> SELECT spark_catalog.default.func(4, 3); + 6 +``` diff --git a/docs/sql-ref-scripting.md b/docs/sql-ref-scripting.md new file mode 100644 index 0000000000000..a8bccb471d563 --- /dev/null +++ b/docs/sql-ref-scripting.md @@ -0,0 +1,89 @@ +--- +layout: global +title: SQL Scripting +displayTitle: SQL Scripting +license: | + 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. +--- + +You can employ powerful procedural logic using SQL/PSM standard-based scripting syntax. +Any SQL script consists of and starts with a [compound statement](control-flow/compound-stmt.html) block (`BEGIN ... END`). +A compound statement starts with a section to declare local variables, user-defined conditions, and condition handlers, which are used to catch exceptions. +This is followed by the compound statement body, which consists of: + +- Flow control statements include loops over predicate expressions, [FOR](control-flow/for-stmt.html) loops over query results, conditional logic such as [IF](control-flow/if-stmt.html) and [CASE](control-flow/case-stmt.html), and means to break out loops such as [LEAVE](control-flow/leave-stmt.html) and [ITERATE](control-flow/iterate-stmt.html). +- DDL statements such as `ALTER`, `CREATE`, `DROP`. +- DML statements [INSERT](sql-ref-syntax-dml-insert-into.html). +- [Queries](sql-ref-syntax-qry-select.html) that return result sets to the invoker of the script. +- [SET](sql-ref-syntax-aux-set-var.html) statements to set local variables as well as session variables. +- The [EXECUTE IMMEDIATE](sql-ref-syntax-aux-exec-imm.html) statement. +- Nested compound statements, which provide nested scopes for variables, conditions, and condition handlers. + +## Enablement + +To enable SQL Scripting, set the `spark.sql.scripting.enabled` flag to `true`. + +## Passing data between the invoker and the compound statement + +There are two ways to pass data to and from a SQL script: + +- Use session variables to pass scalar values or small sets of arrays or maps from one SQL script to another. +- Use parameter markers to pass scalar values or small sets of arrays or map data from a notebook widget, Python, or another language to the SQL Script. + +## Variable scoping + +Variables declared within a compound statement can be referenced in any expression within a compound statement. +Spark resolves identifiers from the innermost scope outward, following the rules described in [Name Resolution](sql-ref-name-resolution.html). +You can use the optional compound statement labels to disambiguate duplicate variable names. + +## Condition handling + +SQL Scripting supports condition handlers, which are used to intercept and process exceptions to `EXIT` processing of the SQL script. + +Condition handlers can be defined to handle three distinct classes of conditions: + +- One or more named conditions that can be a specific Spark-defined error class such as `DIVIDE_BY_ZERO` or a user-declared condition. + These handlers handle these specific conditions. + +- One or more `SQLSTATE`s, that can be raised by Spark. + These handlers can handle any condition associated with that `SQLSTATE`. + +- A generic `SQLEXCEPTION` handler can catch all conditions falling into the `SQLEXCEPTION` (any `SQLSTATE` which is not `XX***` and not `02***`). + +The following are used to decide which condition handler applies to an exception. +This condition handler is called the **most appropriate handler**: + +- A condition handler cannot apply to any statement defined in its own body or the body of any condition handler declared in the same compound statement. + +- The applicable condition handlers defined in the innermost compound statement within which the exception was raised are appropriate. + +- If more than one appropriate handler is available, the most specific handler is the most appropriate. + For example, a handler on a named condition is more specific than one on a named `SQLSTATE`. + A generic `EXCEPTION` handler is the least specific. + +The outcome of a condition handler is to execute the statement following the compound statement that declared the handler to execute next. + +The following is a list of supported control flow statements: + +* [CASE](control-flow/case-stmt.html) +* [compound statement](control-flow/compound-stmt.html) +* [FOR](control-flow/for-stmt.html) +* [IF](control-flow/if-stmt.html) +* [ITERATE](control-flow/iterate-stmt.html) +* [LEAVE](control-flow/leave-stmt.html) +* [LOOP](control-flow/loop-stmt.html) +* [REPEAT](control-flow/repeat-stmt.html) +* [WHILE](control-flow/while-stmt.html) diff --git a/docs/sql-ref-syntax-qry-select-tvf.md b/docs/sql-ref-syntax-qry-select-tvf.md index 099f423f18531..9dd73a3f25263 100644 --- a/docs/sql-ref-syntax-qry-select-tvf.md +++ b/docs/sql-ref-syntax-qry-select-tvf.md @@ -172,5 +172,5 @@ SELECT * FROM test, LATERAL explode (ARRAY(3,4)) AS c2; ### Related Statements * [SELECT](sql-ref-syntax-qry-select.html) -* [LATERAL](sql-ref-syntax-qry-select-lateral-subquery.md) +* [LATERAL](sql-ref-syntax-qry-select-lateral-subquery.html) * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index 3dc7d47c4f454..0a2b9ba34b522 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -90,6 +90,20 @@ ability to generate logical and physical plan for a given query using * [star (*) Clause](sql-ref-syntax-qry-star.html) * [EXPLAIN](sql-ref-syntax-qry-explain.html) +### SQL Scripting Statements + +You use SQL scripting to execute procedural logic in SQL. + +* [CASE](control-flow/case-stmt.html) +* [compound statement](control-flow/compound-stmt.html) +* [FOR](control-flow/for-stmt.html) +* [IF](control-flow/if-stmt.html) +* [ITERATE](control-flow/iterate-stmt.html) +* [LEAVE](control-flow/leave-stmt.html) +* [LOOP](control-flow/loop-stmt.html) +* [REPEAT](control-flow/repeat-stmt.html) +* [WHILE](control-flow/while-stmt.html) + ### Auxiliary Statements * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) diff --git a/docs/sql-ref.md b/docs/sql-ref.md index 6d557caaca3d6..cf9a887bd4928 100644 --- a/docs/sql-ref.md +++ b/docs/sql-ref.md @@ -37,9 +37,12 @@ Spark SQL is Apache Spark's module for working with structured data. This guide * [IDENTIFIER clause](sql-ref-identifier-clause.html) * [Literals](sql-ref-literals.html) * [Null Semantics](sql-ref-null-semantics.html) + * [Name Resolution](sql-ref-name-resolution.html) + * [SQL Scripting](sql-ref-scripting.html) * [SQL Syntax](sql-ref-syntax.html) * [DDL Statements](sql-ref-syntax.html#ddl-statements) * [DML Statements](sql-ref-syntax.html#dml-statements) * [Data Retrieval Statements](sql-ref-syntax.html#data-retrieval-statements) + * [SQL Scripting Statements](sql-ref-syntax.html#sql-scripting-statements) * [Auxiliary Statements](sql-ref-syntax.html#auxiliary-statements) * [Pipe Syntax](sql-pipe-syntax.html) From 2836f5a5c3eb89d7e042a7ea97ca98014cd8e9f8 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Mon, 24 Nov 2025 15:44:23 -0800 Subject: [PATCH 212/400] [SPARK-54289][SQL][FOLLOW-UP] Make Merge Into update assignment by field default for UPDATE SET * and align configs ### What changes were proposed in this pull request? Follow up of: https://github.com/apache/spark/pull/53149 1. Make the update assignment by field the Spark 4.1 behavior. For context, the case to allow assignment key and value to be different struct for MERGE INTO is new in Spark 4.1 so we have a chance to define the behavior. In Spark, nested fields are usually treated as top level column so it should follow the behavior: see https://github.com/apache/spark/pull/53149#discussion_r2557262463 2. Rename existing config to control the struct type compatibility check in assignment. We do not need to mention 'source' as actually the assignment can be to anything, not necessarily to source table. ### Why are the changes needed? See above ### Does this PR introduce _any_ user-facing change? No, this feature is unreleased (allowing assignment source to be of different struct type as target) ### How was this patch tested? Existing unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #53199 from szehon-ho/merge_schema_evolution_update_nested_follow. Authored-by: Szehon Ho Signed-off-by: Dongjoon Hyun (cherry picked from commit 9846dd826e16ec5c125f6e2e708b18d2a61fe8a9) Signed-off-by: Dongjoon Hyun --- .../catalyst/analysis/AssignmentUtils.scala | 3 +- .../ResolveRowLevelCommandAssignments.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 17 +- .../connector/MergeIntoTableSuiteBase.scala | 917 ++++++++---------- 4 files changed, 428 insertions(+), 511 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala index 6cbc17c673817..185dc5ec54f6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getDefaultValueExprOrNullLit import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -183,7 +182,7 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { } else if (exactAssignments.isEmpty && fieldAssignments.isEmpty) { TableOutputResolver.checkNullability(colExpr, col, conf, colPath) } else if (exactAssignments.nonEmpty) { - if (SQLConf.get.mergeUpdateStructsByField && updateStar) { + if (updateStar) { val value = exactAssignments.head.value col.dataType match { case structType: StructType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala index 93ef98e3183af..d1b8eab131916 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala @@ -53,7 +53,7 @@ object ResolveRowLevelCommandAssignments extends Rule[LogicalPlan] { case m: MergeIntoTable if !m.skipSchemaResolution && m.resolved && m.rewritable && !m.aligned && !m.needSchemaEvolution => validateStoreAssignmentPolicy() - val coerceNestedTypes = SQLConf.get.mergeCoerceNestedTypes + val coerceNestedTypes = SQLConf.get.coerceMergeNestedTypes m.copy( targetTable = cleanAttrMetadata(m.targetTable), matchedActions = alignActions(m.targetTable.output, m.matchedActions, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9633011cf872e..18cd916cb5d24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6620,18 +6620,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD = - buildConf("spark.sql.merge.nested.type.assign.by.field") - .internal() - .doc("If enabled and spark.sql.merge.source.nested.type.coercion.enabled is true," + - "allow MERGE INTO with UPDATE SET * action to set nested structs field by field. " + - "In updated rows, target structs will preserve the original value for fields missing " + - "in the the source struct. If disabled, the entire target struct will be replaced, " + - "and fields missing in the source struct will be null.") - .version("4.1.0") - .booleanConf - .createWithDefault(true) - /** * Holds information about keys that have been deprecated. * @@ -7789,12 +7777,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def legacyXMLParserEnabled: Boolean = getConf(SQLConf.LEGACY_XML_PARSER_ENABLED) - def mergeCoerceNestedTypes: Boolean = + def coerceMergeNestedTypes: Boolean = getConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED) - def mergeUpdateStructsByField: Boolean = - getConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index 7051a0b455e32..b7a8ff374b841 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -3231,180 +3231,161 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase test("merge into schema evolution replace column with nested struct and set all columns") { Seq(true, false).foreach { withSchemaEvolution => - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { - withTempView("source") { - // Create table using Spark SQL - sql( - s"""CREATE TABLE $tableNameAsString ( - |pk INT NOT NULL, - |s STRUCT, m: MAP>>, - |dep STRING) - |PARTITIONED BY (dep) - |""".stripMargin) - - // Insert data using DataFrame API with objects - val tableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", ArrayType(IntegerType)), - StructField("m", MapType(StringType, StringType)) - ))) - ))), - StructField("dep", StringType) - )) - val targetData = Seq( - Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") - ) - spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) - .coalesce(1).writeTo(tableNameAsString).append() + withTempView("source") { + // Create table using Spark SQL + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + // Insert data using DataFrame API with objects + val tableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) + .coalesce(1).writeTo(tableNameAsString).append() - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - // missing column 'a' - StructField("m", MapType(StringType, StringType)), - StructField("c3", BooleanType) // new column - ))) - ))), - StructField("dep", StringType) - )) - val sourceData = Seq( - Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), - Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) - .createOrReplaceTempView("source") + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // missing column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + StructField("dep", StringType) + )) + val sourceData = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source") - val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" - val mergeStmt = - s"""MERGE $schemaEvolutionClause - |INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin - if (withSchemaEvolution) { - sql(mergeStmt) - if (updateByFields) { - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(Seq(1, 2), Map("c" -> "d"), false)), "sales"), - Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) - } else { - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "sales"), - Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) - } - } else { - val exception = intercept[org.apache.spark.sql.AnalysisException] { - sql(mergeStmt) - } - assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") - assert(exception.getMessage.contains( - "Cannot write extra fields `c3` to the struct `s`.`c2`")) - } + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + if (withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(Seq(1, 2), Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) } } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } test("merge into schema evolution replace column with nested struct and update " + "top level struct") { Seq(true, false).foreach { withSchemaEvolution => - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { - withTempView("source") { - // Create table using Spark SQL - sql( - s"""CREATE TABLE $tableNameAsString ( - |pk INT NOT NULL, - |s STRUCT, m: MAP>>, - |dep STRING) - |PARTITIONED BY (dep) - |""".stripMargin) + withTempView("source") { + // Create table using Spark SQL + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) - // Insert data using DataFrame API with objects - val tableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", ArrayType(IntegerType)), - StructField("m", MapType(StringType, StringType)) - ))) - ))), - StructField("dep", StringType) - )) - val targetData = Seq( - Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") - ) - spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) - .coalesce(1).writeTo(tableNameAsString).append() + // Insert data using DataFrame API with objects + val tableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) + .coalesce(1).writeTo(tableNameAsString).append() - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - // missing column 'a' - StructField("m", MapType(StringType, StringType)), - StructField("c3", BooleanType) // new column - ))) - ))), - StructField("dep", StringType) - )) - val sourceData = Seq( - Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), - Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") - ) - spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) - .createOrReplaceTempView("source") + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // missing column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + StructField("dep", StringType) + )) + val sourceData = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source") - val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" - val mergeStmt = - s"""MERGE $schemaEvolutionClause - |INTO $tableNameAsString t - |USING source src - |ON t.pk = src.pk - |WHEN MATCHED THEN - | UPDATE SET s = src.s - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin - if (withSchemaEvolution) { - sql(mergeStmt) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "hr"), - Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) - } else { - val exception = intercept[org.apache.spark.sql.AnalysisException] { - sql(mergeStmt) - } - assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") - assert(exception.getMessage.contains( - "Cannot write extra fields `c3` to the struct `s`.`c2`")) - } + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t + |USING source src + |ON t.pk = src.pk + |WHEN MATCHED THEN + | UPDATE SET s = src.s + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + if (withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "hr"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) } } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } @@ -4661,124 +4642,163 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } test("merge with null struct") { - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { - withTempView("source") { - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT, - |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" - .stripMargin) + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) - // Source table matches target table schema - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StringType) - ))), - StructField("dep", StringType) - )) + // Source table matches target table schema + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) - val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") - sql( - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, "a"), "sales"), - Row(1, null, "engineering"), - Row(2, null, "finance"))) - } - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a"), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - test("merge with null struct - update field") { - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { - withTempView("source") { - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT, - |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" - .stripMargin) - // Source table matches target table schema - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StringType) - ))), - StructField("dep", StringType) - )) + test("merge with null struct - update field") { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) - val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") + // Source table matches target table schema + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) - sql( - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET s = source.s - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, "a"), "sales"), - Row(1, null, "hr"), - Row(2, null, "finance"))) - } - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET s = source.s + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a"), "sales"), + Row(1, null, "hr"), + Row(2, null, "finance"))) } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } test("merge with null struct into non-nullable struct column") { - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT NOT NULL, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) + + // Source table has null for the struct column + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + // Should throw an exception when trying to insert/update null into NOT NULL column + val exception = intercept[Exception] { + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + } + assert(exception.getMessage.contains( + "NULL value appeared in non-nullable field")) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge with with null struct with missing nested field") { + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { withTempView("source") { + // Target table has nested struct with fields c1 and c2 createAndInitTable( s"""pk INT NOT NULL, - |s STRUCT NOT NULL, + |s STRUCT>, |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" .stripMargin) - // Source table has null for the struct column + // Source table has null for the nested struct val sourceTableSchema = StructType(Seq( StructField("pk", IntegerType), StructField("s", StructType(Seq( StructField("c1", IntegerType), - StructField("c2", StringType) + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) ))), StructField("dep", StringType) )) @@ -4790,107 +4810,45 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") - // Should throw an exception when trying to insert/update null into NOT NULL column - val exception = intercept[Exception] { - sql( - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) - } - assert(exception.getMessage.contains( - "NULL value appeared in non-nullable field")) - } - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") - } - } - - test("merge with with null struct with missing nested field") { - Seq(true, false).foreach { updateByFields => - Seq(true, false).foreach { coerceNestedTypes => - withSQLConf( - SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString, - SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> - coerceNestedTypes.toString) { - withTempView("source") { - // Target table has nested struct with fields c1 and c2 - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT>, - |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" - .stripMargin) - - // Source table has null for the nested struct - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType) - // missing field 'b' - ))) - ))), - StructField("dep", StringType) - )) - - val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") - - val mergeStmt = - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin + val mergeStmt = + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin - if (coerceNestedTypes) { + if (coerceNestedTypes) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } else { + // Without coercion, the merge should fail due to missing field + val exception = intercept[org.apache.spark.sql.AnalysisException] { sql(mergeStmt) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, null, "engineering"), - Row(2, null, "finance"))) - } else { - // Without coercion, the merge should fail due to missing field - val exception = intercept[org.apache.spark.sql.AnalysisException] { - sql(mergeStmt) - } - assert(exception.errorClass.get == - "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") - assert(exception.getMessage.contains( - "Cannot write incompatible data for the table ``: " + - "Cannot find data for the output column `s`.`c2`.`b`.")) } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot find data for the output column `s`.`c2`.`b`.")) } } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } test("merge null struct with schema evolution - source with missing and extra nested fields") { - Seq(true, false).foreach { updateByFields => - Seq(true, false).foreach { withSchemaEvolution => - Seq(true, false).foreach { coerceNestedTypes => - withSQLConf( - SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString, - SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> - coerceNestedTypes.toString) { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { withTempView("source") { // Target table has nested struct with fields c1 and c2 createAndInitTable( @@ -4967,17 +4925,14 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") - } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } } test("merge null struct with non-nullable nested field - source with missing " + "and extra nested fields") { - withSQLConf( - SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> "true", SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> "true") { withTempView("source") { // Target table has nested struct with NON-NULLABLE field b @@ -5033,153 +4988,142 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } test("merge with null struct using default value") { - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { - withTempView("source") { - // Target table has nested struct with a default value - sql( - s"""CREATE TABLE $tableNameAsString ( - | pk INT NOT NULL, - | s STRUCT> DEFAULT - | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), - | dep STRING) - |PARTITIONED BY (dep) - |""".stripMargin) + withTempView("source") { + // Target table has nested struct with a default value + sql( + s"""CREATE TABLE $tableNameAsString ( + | pk INT NOT NULL, + | s STRUCT> DEFAULT + | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), + | dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) - // Insert initial data using DataFrame API - val initialSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType), - StructField("b", StringType) - ))) - ))), - StructField("dep", StringType) - )) - val initialData = Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, Row(2, Row(20, "y")), "hr") - ) - spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) - .writeTo(tableNameAsString).append() + // Insert initial data using DataFrame API + val initialSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + val initialData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) + .writeTo(tableNameAsString).append() - // Source table has null for the nested struct - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType) - // missing field 'b' - ))) - ))), - StructField("dep", StringType) - )) + // Source table has null for the nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) + ))), + StructField("dep", StringType) + )) - val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") - sql( - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, null, "engineering"), - Row(2, null, "finance"))) - } - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } + test("merge with source missing struct column with default value") { - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key -> - updateByFields.toString) { - withTempView("source") { - // Target table has nested struct with a default value - sql( - s"""CREATE TABLE $tableNameAsString ( - | pk INT NOT NULL, - | s STRUCT> DEFAULT - | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), - | dep STRING) - |PARTITIONED BY (dep) - |""".stripMargin) + withTempView("source") { + // Target table has nested struct with a default value + sql( + s"""CREATE TABLE $tableNameAsString ( + | pk INT NOT NULL, + | s STRUCT> DEFAULT + | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), + | dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) - // Insert initial data using DataFrame API - val initialSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType), - StructField("b", StringType) - ))) - ))), - StructField("dep", StringType) - )) - val initialData = Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, Row(2, Row(20, "y")), "hr") - ) - spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) - .writeTo(tableNameAsString).append() + // Insert initial data using DataFrame API + val initialSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + val initialData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) + .writeTo(tableNameAsString).append() - // Source table is completely missing the struct column 's' - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("dep", StringType) - )) + // Source table is completely missing the struct column 's' + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("dep", StringType) + )) - val data = Seq( - Row(1, "engineering"), - Row(2, "finance") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") + val data = Seq( + Row(1, "engineering"), + Row(2, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + // When inserting without specifying the struct column, default should be used + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET dep = source.dep + |WHEN NOT MATCHED THEN + | INSERT (pk, dep) VALUES (source.pk, source.dep) + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "engineering"), + Row(2, Row(999, Row(999, "default")), "finance"))) - // When inserting without specifying the struct column, default should be used - sql( - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET dep = source.dep - |WHEN NOT MATCHED THEN - | INSERT (pk, dep) VALUES (source.pk, source.dep) - |""".stripMargin) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, Row(2, Row(20, "y")), "engineering"), - Row(2, Row(999, Row(999, "default")), "finance"))) - } - } sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } test("merge into with source missing fields in nested struct") { Seq(true, false).foreach { nestedTypeCoercion => - Seq(true, false).foreach { updateByFields => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_UPDATE_BY_FIELD.key - -> updateByFields.toString, - SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> nestedTypeCoercion.toString) { withTempView("source") { // Target table has nested struct: s.c1, s.c2.a, s.c2.b @@ -5221,21 +5165,11 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase if (nestedTypeCoercion) { sql(mergeStmt) - if (updateByFields) { - // When updating by fields, only non-null fields are updated - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(20, true)), "sales"), - Row(2, Row(20, Row(30, false)), "engineering"))) - } else { - // When updating by top level column, the missing field is set to NULL - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(1, Row(10, Row(20, null)), "sales"), - Row(2, Row(20, Row(30, null)), "engineering"))) - } + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(20, true)), "sales"), + Row(2, Row(20, Row(30, false)), "engineering"))) } else { val exception = intercept[Exception] { sql(mergeStmt) @@ -5247,7 +5181,6 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } - } } test("merge with named_struct missing non-nullable field") { From f7027f1f9ec36bef8aa9b9614aa41e62d5147afe Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 24 Nov 2025 16:16:04 -0800 Subject: [PATCH 213/400] [SPARK-54488][BUILD] Fix Connect JVM client leaks `protobuf-java-util` in shading rules ### What changes were proposed in this pull request? Fix Connect JVM client leaks `protobuf-java-util` in shading rules by setting `protobuf-java-util` to `compile` scope (was `provided`) ### Why are the changes needed? A bug fix, `spark-connect-client-jvm` jar should correctly shade all protobuf classes, same as the `spark-connect` server jar does. ### Does this PR introduce _any_ user-facing change? Yes, users who pull `spark-connect-client-jvm` as dependencies, will not see transitive `protobuf-java-util` dependency. ### How was this patch tested? ``` build/mvn clean install -DskipTests -pl sql/connect/client/jvm -am ``` ```patch [INFO] --- shade:3.6.0:shade (default) spark-connect-client-jvm_2.13 --- [INFO] Including org.apache.spark:spark-connect-common_2.13:jar:4.2.0-SNAPSHOT in the shaded jar. [INFO] Including io.grpc:grpc-netty:jar:1.76.0 in the shaded jar. [INFO] Including io.grpc:grpc-api:jar:1.76.0 in the shaded jar. [INFO] Including org.codehaus.mojo:animal-sniffer-annotations:jar:1.24 in the shaded jar. [INFO] Including io.grpc:grpc-core:jar:1.76.0 in the shaded jar. [INFO] Including com.google.android:annotations:jar:4.1.1.4 in the shaded jar. [INFO] Including io.grpc:grpc-context:jar:1.76.0 in the shaded jar. [INFO] Including io.perfmark:perfmark-api:jar:0.27.0 in the shaded jar. [INFO] Including io.grpc:grpc-util:jar:1.76.0 in the shaded jar. [INFO] Including io.grpc:grpc-protobuf:jar:1.76.0 in the shaded jar. [INFO] Including com.google.api.grpc:proto-google-common-protos:jar:2.59.2 in the shaded jar. [INFO] Including io.grpc:grpc-protobuf-lite:jar:1.76.0 in the shaded jar. [INFO] Including io.grpc:grpc-services:jar:1.76.0 in the shaded jar. [INFO] Including io.grpc:grpc-stub:jar:1.76.0 in the shaded jar. [INFO] Including io.grpc:grpc-inprocess:jar:1.76.0 in the shaded jar. [INFO] Including io.netty:netty-codec-http2:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-common:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-buffer:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-transport:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-resolver:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec-base:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-handler:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec-http:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec-compression:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-handler-proxy:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec-socks:jar:4.2.7.Final in the shaded jar. [INFO] Including io.netty:netty-transport-native-unix-common:jar:4.2.7.Final in the shaded jar. [INFO] Including org.apache.spark:spark-sql-api_2.13:jar:4.2.0-SNAPSHOT in the shaded jar. [INFO] Including org.apache.arrow:arrow-vector:jar:18.3.0 in the shaded jar. [INFO] Including org.apache.arrow:arrow-format:jar:18.3.0 in the shaded jar. [INFO] Including org.apache.arrow:arrow-memory-core:jar:18.3.0 in the shaded jar. [INFO] Including com.google.flatbuffers:flatbuffers-java:jar:25.2.10 in the shaded jar. [INFO] Including org.apache.arrow:arrow-memory-netty:jar:18.3.0 in the shaded jar. [INFO] Including org.apache.arrow:arrow-memory-netty-buffer-patch:jar:18.3.0 in the shaded jar. [INFO] Including com.google.protobuf:protobuf-java:jar:4.33.0 in the shaded jar. + [INFO] Including com.google.protobuf:protobuf-java-util:jar:4.33.0 in the shaded jar. [INFO] Including com.google.code.gson:gson:jar:2.11.0 in the shaded jar. [INFO] Including com.google.guava:guava:jar:33.4.8-jre in the shaded jar. [INFO] Including com.google.guava:listenablefuture:jar:9999.0-empty-to-avoid-conflict-with-guava in the shaded jar. [INFO] Including com.google.guava:failureaccess:jar:1.0.3 in the shaded jar. ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53191 from pan3793/SPARK-54488. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 21b61660bcd00240f4c250f4a64fc544bc4c6117) Signed-off-by: Dongjoon Hyun --- sql/connect/client/jvm/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 0262d1af9fdb4..40621d51d7dcb 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -74,6 +74,11 @@ protobuf-java compile + + com.google.protobuf + protobuf-java-util + compile + com.google.guava guava From 5b1ce6d502beb7ed85051e1140cc6dc08de2ec9b Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 24 Nov 2025 17:42:43 -0800 Subject: [PATCH 214/400] [SPARK-54491][SQL] Fix insert into temp view on DSv2 table failure ### What changes were proposed in this pull request? Resolve `V2TableReference` for table in `InsertIntoStatement`. ### Why are the changes needed? https://github.com/apache/spark/pull/52876 brought in `V2TableReference` which broke relation resolution for insert into temp view on DSv2 table. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53196 from manuzhang/FIX-SPARK-54491. Authored-by: manuzhang Signed-off-by: Dongjoon Hyun (cherry picked from commit 3f5a2b9fc3344b397320c8f507dd1ab9068e99d5) Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/analysis/Analyzer.scala | 2 ++ .../spark/sql/connector/DataSourceV2SQLSuite.scala | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3b916a62b3886..08c31939f161a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1207,6 +1207,8 @@ class Analyzer( val relation = table match { case u: UnresolvedRelation if !u.isStreaming => resolveRelation(u).getOrElse(u) + case r: V2TableReference => + relationResolution.resolveReference(r) case other => other } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 847af570d6f39..5a9494cf81730 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -3986,6 +3986,19 @@ class DataSourceV2SQLSuiteV1Filter } } + test("SPARK-54491: insert into temp view on DSv2 table") { + val t = s"testcat.default.t" + val v = "v" + withTable(t) { + withTempView(v) { + sql(s"CREATE TABLE $t (id INT) USING foo") + spark.table(t).createOrReplaceTempView(v) + sql(s"INSERT INTO v VALUES (1)") + checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(1))) + } + } + } + private def testNotSupportedV2Command( sqlCommand: String, sqlParams: String, From 0c986b3224a67255580af563e144a433fd1a9d0c Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 25 Nov 2025 14:08:29 +0900 Subject: [PATCH 215/400] [SPARK-54477][CONNECT][PYTHON][TESTS] Mark tests to reeanble for 4.0 client <> master server job ### What changes were proposed in this pull request? This PR proposes to skip the tests that fails in 4.0 client <> master server job. It marks them with JIRA IDs. ### Why are the changes needed? To enable the build https://github.com/apache/spark/pull/53188 ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? [Manually ran the tests in my fork.](https://github.com/HyukjinKwon/spark/actions/runs/19655913777) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53204 from HyukjinKwon/SPARK-54477. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 8f747eef9a38b62d63cb5983bc364f1952aee0d7) Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/tests/test_typedef.py | 4 ++++ python/pyspark/sql/streaming/readwriter.py | 7 ++++--- ...st_parity_pandas_grouped_map_with_state.py | 4 ++++ .../streaming/test_parity_foreach_batch.py | 4 ++++ .../tests/pandas/test_pandas_cogrouped_map.py | 4 ++++ .../tests/pandas/test_pandas_grouped_map.py | 19 +++++++++++++++++++ .../sql/tests/pandas/test_pandas_map.py | 3 +++ .../tests/test_python_streaming_datasource.py | 4 ++++ 8 files changed, 46 insertions(+), 3 deletions(-) diff --git a/python/pyspark/pandas/tests/test_typedef.py b/python/pyspark/pandas/tests/test_typedef.py index 329af01a39440..29d1c170f0658 100644 --- a/python/pyspark/pandas/tests/test_typedef.py +++ b/python/pyspark/pandas/tests/test_typedef.py @@ -15,6 +15,7 @@ # limitations under the License. # +import os import sys import unittest import datetime @@ -310,6 +311,9 @@ def f() -> ps.Series[pdf.a.dtype]: self.assertRaisesRegex(TypeError, "object.*not understood", try_infer_return_type) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54495: To be reenabled" + ) def test_as_spark_type_pandas_on_spark_dtype(self): type_mapper = { # binary diff --git a/python/pyspark/sql/streaming/readwriter.py b/python/pyspark/sql/streaming/readwriter.py index 8121dd609950e..4d36846b74b5b 100644 --- a/python/pyspark/sql/streaming/readwriter.py +++ b/python/pyspark/sql/streaming/readwriter.py @@ -1568,6 +1568,7 @@ def foreach(self, f: Union[Callable[[Row], None], "SupportsProcess"]) -> "DataSt self._jwrite.foreach(jForeachWriter) return self + # SPARK-54478: Reenable doctest def foreachBatch(self, func: Callable[["DataFrame", int], None]) -> "DataStreamWriter": """ Sets the output of the streaming query to be processed using the provided @@ -1600,9 +1601,9 @@ def foreachBatch(self, func: Callable[["DataFrame", int], None]) -> "DataStreamW ... my_value = 100 ... batch_df.collect() ... - >>> q = df.writeStream.foreachBatch(func).start() - >>> time.sleep(3) - >>> q.stop() + >>> q = df.writeStream.foreachBatch(func).start() # doctest: +SKIP + >>> time.sleep(3) # doctest: +SKIP + >>> q.stop() # doctest: +SKIP >>> # if in Spark Connect, my_value = -1, else my_value = 100 """ from py4j.java_gateway import java_import diff --git a/python/pyspark/sql/tests/connect/pandas/streaming/test_parity_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/connect/pandas/streaming/test_parity_pandas_grouped_map_with_state.py index 8e60f8a168dff..18f9d4e9dcf45 100644 --- a/python/pyspark/sql/tests/connect/pandas/streaming/test_parity_pandas_grouped_map_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/streaming/test_parity_pandas_grouped_map_with_state.py @@ -15,6 +15,7 @@ # limitations under the License. # import unittest +import os from pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state import ( GroupedApplyInPandasWithStateTestsMixin, @@ -22,6 +23,9 @@ from pyspark.testing.connectutils import ReusedConnectTestCase +@unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54479: To be reenabled" +) class GroupedApplyInPandasWithStateTests( GroupedApplyInPandasWithStateTestsMixin, ReusedConnectTestCase ): diff --git a/python/pyspark/sql/tests/connect/streaming/test_parity_foreach_batch.py b/python/pyspark/sql/tests/connect/streaming/test_parity_foreach_batch.py index 632fa4628d1b6..c2a747dd57f36 100644 --- a/python/pyspark/sql/tests/connect/streaming/test_parity_foreach_batch.py +++ b/python/pyspark/sql/tests/connect/streaming/test_parity_foreach_batch.py @@ -16,6 +16,7 @@ # import unittest +import os from pyspark.sql.tests.streaming.test_streaming_foreach_batch import StreamingTestsForeachBatchMixin from pyspark.testing.connectutils import ReusedConnectTestCase, should_test_connect @@ -25,6 +26,9 @@ from pyspark.errors.exceptions.connect import StreamingPythonRunnerInitializationException +@unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54480: To be reenabled" +) class StreamingForeachBatchParityTests(StreamingTestsForeachBatchMixin, ReusedConnectTestCase): def test_streaming_foreach_batch_propagates_python_errors(self): super().test_streaming_foreach_batch_propagates_python_errors() diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py index 0d91da0354979..0a77334ff85ec 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py @@ -18,6 +18,7 @@ import unittest import logging from typing import cast +import os from pyspark.sql import functions as sf from pyspark.sql.functions import pandas_udf, udf @@ -243,6 +244,9 @@ def merge_pandas(lft, rgt): self._test_merge_empty(fn=merge_pandas) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54481: To be reenabled" + ) def test_apply_in_pandas_returning_incompatible_type(self): with self.quiet(): self.check_apply_in_pandas_returning_incompatible_type() diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index a86ef3e4f3f32..f8c6469338090 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -18,6 +18,7 @@ import datetime import unittest import logging +import os from collections import OrderedDict from decimal import Decimal @@ -287,12 +288,21 @@ def stats_with_no_column_names(key, pdf): # columns must be in order of applyInPandas schema when no columns given return pd.DataFrame([key + (pdf.v.mean(),)]) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" + ) def test_apply_in_pandas_returning_column_names(self): self._test_apply_in_pandas(ApplyInPandasTestsMixin.stats_with_column_names) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" + ) def test_apply_in_pandas_returning_no_column_names(self): self._test_apply_in_pandas(ApplyInPandasTestsMixin.stats_with_no_column_names) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" + ) def test_apply_in_pandas_returning_column_names_sometimes(self): def stats(key, pdf): if key[0] % 2: @@ -332,9 +342,15 @@ def check_apply_in_pandas_returning_no_column_names_and_wrong_amount(self): lambda key, pdf: pd.DataFrame([key + (pdf.v.mean(), pdf.v.std())]) ) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" + ) def test_apply_in_pandas_returning_empty_dataframe(self): self._test_apply_in_pandas_returning_empty_dataframe(pd.DataFrame()) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" + ) def test_apply_in_pandas_returning_incompatible_type(self): with self.quiet(): self.check_apply_in_pandas_returning_incompatible_type() @@ -887,6 +903,9 @@ def stats(key, pdf): for row in result: self.assertEqual(24.5, row[1]) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" + ) def _test_apply_in_pandas_returning_empty_dataframe_error(self, empty_df, error): with self.quiet(): with self.assertRaisesRegex(PythonException, error): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_map.py b/python/pyspark/sql/tests/pandas/test_pandas_map.py index 946d56f2fe637..f816b09fe5ae4 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_map.py @@ -273,6 +273,9 @@ def test_dataframes_with_more_columns(self): actual = df.repartition(1).mapInPandas(f, "id long, value long").collect() self.assertEqual(actual, expected) + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54483: To be reenabled" + ) def test_dataframes_with_incompatible_types(self): with self.quiet(): self.check_dataframes_with_incompatible_types() diff --git a/python/pyspark/sql/tests/test_python_streaming_datasource.py b/python/pyspark/sql/tests/test_python_streaming_datasource.py index 9879231540f1d..4130b66fce655 100644 --- a/python/pyspark/sql/tests/test_python_streaming_datasource.py +++ b/python/pyspark/sql/tests/test_python_streaming_datasource.py @@ -139,6 +139,9 @@ def streamWriter(self, schema, overwrite): return TestDataSource + @unittest.skipIf( + os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54484: To be reenabled" + ) def test_stream_reader(self): self.spark.dataSource.register(self._get_test_data_source()) df = self.spark.readStream.format("TestDataSource").load() @@ -213,6 +216,7 @@ def streamReader(self, schema): assertDataFrameEqual(df, expected_data) + @unittest.skipIf(os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "To be reenabled") def test_simple_stream_reader(self): class SimpleStreamReader(SimpleDataSourceStreamReader): def initialOffset(self): From 9618d0b593baa612a50395fa09a55d2ba242565d Mon Sep 17 00:00:00 2001 From: Ganesha S Date: Tue, 25 Nov 2025 21:06:55 +0800 Subject: [PATCH 216/400] [SPARK-54130][SQL][FOLLOWUP] Refactor tests to use withEmptyCatalog for consistency ### What changes were proposed in this pull request? This PR refactors two tests in `SessionCatalogSuite.scala` to use the `withEmptyCatalog` helper method instead of manually creating a `SessionCatalog` instance with `newEmptyCatalog()`: - "UnresolvedCatalogRelation requires database in identifier" - "HiveTableRelation requires database in identifier" ### Why are the changes needed? **Consistency**: All other tests in `SessionCatalogSuite` use the `withEmptyCatalog` or `withBasicCatalog` helper pattern. These two tests, added in SPARK-54130, were using manual catalog initialization, which is inconsistent with the rest of the test suite. ### Does this PR introduce _any_ user-facing change? No. This is a test-only refactoring with no functional changes. ### How was this patch tested? - Verified no linter errors in the modified file - The test logic remains identical, only the catalog initialization pattern changed - Both tests validate the same assertion error messages as before - The `withEmptyCatalog` helper handles catalog creation and cleanup automatically ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.5 Closes #53208 from ganeshashree/SPARK-54130-2. Authored-by: Ganesha S Signed-off-by: Wenchen Fan (cherry picked from commit 211dd995b221f135340375159672dcb77ef90ef4) Signed-off-by: Wenchen Fan --- .../catalog/SessionCatalogSuite.scala | 102 +++++++++--------- 1 file changed, 51 insertions(+), 51 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index a1b7113d79478..7a0a37f380992 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -2095,68 +2095,68 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { } test("UnresolvedCatalogRelation requires database in identifier") { - val catalog = new SessionCatalog(newEmptyCatalog()) - catalog.createDatabase(newDb("default"), ignoreIfExists = true) - val db = "test_db" - catalog.createDatabase(newDb(db), ignoreIfExists = true) + withEmptyCatalog { catalog => + val db = "test_db" + catalog.createDatabase(newDb(db), ignoreIfExists = true) + + // Create a table with database + val validTable = CatalogTable( + identifier = TableIdentifier("test_table", Some(db)), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("id", IntegerType) + ) + catalog.createTable(validTable, ignoreIfExists = false) - // Create a table with database - val validTable = CatalogTable( - identifier = TableIdentifier("test_table", Some(db)), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType().add("id", IntegerType) - ) - catalog.createTable(validTable, ignoreIfExists = false) + // Try to create UnresolvedCatalogRelation without database - should fail + val tableMetaWithoutDb = validTable.copy( + identifier = TableIdentifier("test_table", None) + ) - // Try to create UnresolvedCatalogRelation without database - should fail - val tableMetaWithoutDb = validTable.copy( - identifier = TableIdentifier("test_table", None) - ) + val exception = intercept[AssertionError] { + UnresolvedCatalogRelation(tableMetaWithoutDb) + } - val exception = intercept[AssertionError] { - UnresolvedCatalogRelation(tableMetaWithoutDb) + val expectedMessage = + "assertion failed: Table identifier `test_table` is missing database name. " + + "UnresolvedCatalogRelation requires a fully qualified table identifier with database." + assert(exception.getMessage === expectedMessage) } - - val expectedMessage = - "assertion failed: Table identifier `test_table` is missing database name. " + - "UnresolvedCatalogRelation requires a fully qualified table identifier with database." - assert(exception.getMessage === expectedMessage) } test("HiveTableRelation requires database in identifier") { - val catalog = new SessionCatalog(newEmptyCatalog()) - catalog.createDatabase(newDb("default"), ignoreIfExists = true) - val db = "test_db" - catalog.createDatabase(newDb(db), ignoreIfExists = true) + withEmptyCatalog { catalog => + val db = "test_db" + catalog.createDatabase(newDb(db), ignoreIfExists = true) + + // Create a table with database + val validTable = CatalogTable( + identifier = TableIdentifier("test_table", Some(db)), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("id", IntegerType) + .add("name", StringType) + ) - // Create a table with database - val validTable = CatalogTable( - identifier = TableIdentifier("test_table", Some(db)), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType() - .add("id", IntegerType) - .add("name", StringType) - ) + // Try to create HiveTableRelation without database - should fail + val tableMetaWithoutDb = validTable.copy( + identifier = TableIdentifier("test_table", None) + ) - // Try to create HiveTableRelation without database - should fail - val tableMetaWithoutDb = validTable.copy( - identifier = TableIdentifier("test_table", None) - ) + val exception = intercept[AssertionError] { + HiveTableRelation( + tableMetaWithoutDb, + Seq(AttributeReference("id", IntegerType)()), + Seq.empty + ) + } - val exception = intercept[AssertionError] { - HiveTableRelation( - tableMetaWithoutDb, - Seq(AttributeReference("id", IntegerType)()), - Seq.empty - ) + val expectedMessage = + "assertion failed: Table identifier `test_table` is missing database name. " + + "HiveTableRelation requires a fully qualified table identifier with database." + assert(exception.getMessage === expectedMessage) } - - val expectedMessage = - "assertion failed: Table identifier `test_table` is missing database name. " + - "HiveTableRelation requires a fully qualified table identifier with database." - assert(exception.getMessage === expectedMessage) } test("SQLFunction requires either exprText or queryText") { From 58a4ad9258312b6e20695230176b0a256a2b199d Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 25 Nov 2025 10:11:37 -0800 Subject: [PATCH 217/400] [SPARK-54444][SQL] Relax DSv2 table checks to restore previous behavior ### What changes were proposed in this pull request? This PR relaxes DSv2 table checks to restore well-known Spark behavior that allowed new fields to be added concurrently without failing the execution of queries (not commands). ### Why are the changes needed? These changes are needed to avoid behavior changes in 4.1. ### Does this PR introduce _any_ user-facing change? Yes, but for UNRELEASED logic. Before this PR, we would fail execution if we detected any schema changes after analysis. After this PR, we would allow new columns to be added in queries and prohibit any schema changes otherwise. ### How was this patch tested? Existing + new tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53202 from aokolnychyi/spark-54444. Authored-by: Anton Okolnychyi Signed-off-by: Gengliang Wang (cherry picked from commit 08273487ec3eee75be3c51ec6bb9f39ab7a01e1a) Signed-off-by: Gengliang Wang --- .../catalog/CatalogV2Implicits.scala | 4 +- .../sql/connector/catalog/CatalogV2Util.scala | 19 +- .../sql/connector/catalog/V2TableUtil.scala | 129 +++++--------- .../apache/spark/sql/util/SchemaUtils.scala | 147 ++++++++++++++- .../connector/catalog/V2TableUtilSuite.scala | 167 ++++++++++++++++-- .../datasources/v2/V2TableRefreshUtil.scala | 62 ++++++- .../v2/WriteToDataSourceV2Exec.scala | 7 +- .../DataSourceV2DataFrameSuite.scala | 131 +++++++++++--- .../connector/MergeIntoDataFrameSuite.scala | 50 ++++++ 9 files changed, 571 insertions(+), 145 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 5cafc42645bec..726527394deb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, ClusterBySpec} import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, QuotingUtils} +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, quoteNameParts, QuotingUtils} import org.apache.spark.sql.connector.expressions.{BucketTransform, ClusterByTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.StructType @@ -223,6 +223,8 @@ private[sql] object CatalogV2Implicits { def quoted: String = parts.map(quoteIfNeeded).mkString(".") + def fullyQuoted: String = quoteNameParts(parts) + def original: String = parts.mkString(".") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 07cb370d18dd9..2d4ef5cd9e07a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -587,12 +587,29 @@ private[sql] object CatalogV2Util { .asTableCatalog } + def toStructType(cols: Seq[MetadataColumn]): StructType = { + StructType(cols.map(toStructField)) + } + + private def toStructField(col: MetadataColumn): StructField = { + val metadata = Option(col.metadataInJSON).map(Metadata.fromJson).getOrElse(Metadata.empty) + var f = StructField(col.name, col.dataType, col.isNullable, metadata) + if (col.comment != null) { + f = f.withComment(col.comment) + } + f + } + + def v2ColumnsToStructType(columns: Array[Column]): StructType = { + v2ColumnsToStructType(columns.toImmutableArraySeq) + } + /** * Converts DS v2 columns to StructType, which encodes column comment and default value to * StructField metadata. This is mainly used to define the schema of v2 scan, w.r.t. the columns * of the v2 table. */ - def v2ColumnsToStructType(columns: Array[Column]): StructType = { + def v2ColumnsToStructType(columns: Seq[Column]): StructType = { StructType(columns.map(v2ColumnToStructField)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala index fa8ed1b21a313..42181c6c8389d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableUtil.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.connector.catalog import java.util.Locale -import scala.collection.mutable - import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, MetadataColumnHelper} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.util.SchemaUtils +import org.apache.spark.sql.util.SchemaValidationMode +import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES import org.apache.spark.util.ArrayImplicits._ private[sql] object V2TableUtil extends SQLConfHelper { @@ -40,10 +40,14 @@ private[sql] object V2TableUtil extends SQLConfHelper { * * @param table the current table metadata * @param relation the relation with captured columns + * @param mode validation mode that defines what changes are acceptable * @return validation errors, or empty sequence if valid */ - def validateCapturedColumns(table: Table, relation: DataSourceV2Relation): Seq[String] = { - validateCapturedColumns(table, relation.table.columns.toImmutableArraySeq) + def validateCapturedColumns( + table: Table, + relation: DataSourceV2Relation, + mode: SchemaValidationMode): Seq[String] = { + validateCapturedColumns(table, relation.table.columns.toImmutableArraySeq, mode) } /** @@ -56,33 +60,16 @@ private[sql] object V2TableUtil extends SQLConfHelper { * * @param table the current table metadata * @param originCols the originally captured columns + * @param mode validation mode that defines what changes are acceptable * @return validation errors, or empty sequence if valid */ - def validateCapturedColumns(table: Table, originCols: Seq[Column]): Seq[String] = { - val errors = mutable.ArrayBuffer[String]() - val colsByNormalizedName = indexColumns(table.columns.toImmutableArraySeq) - val originColsByNormalizedName = indexColumns(originCols) - - originColsByNormalizedName.foreach { case (normalizedName, originCol) => - colsByNormalizedName.get(normalizedName) match { - case Some(col) => - if (originCol.dataType != col.dataType || originCol.nullable != col.nullable) { - val oldType = formatType(originCol.dataType, originCol.nullable) - val newType = formatType(col.dataType, col.nullable) - errors += s"`${originCol.name}` type has changed from $oldType to $newType" - } - case None => - errors += s"${formatColumn(originCol)} has been removed" - } - } - - colsByNormalizedName.foreach { case (normalizedName, col) => - if (!originColsByNormalizedName.contains(normalizedName)) { - errors += s"${formatColumn(col)} has been added" - } - } - - errors.toSeq + def validateCapturedColumns( + table: Table, + originCols: Seq[Column], + mode: SchemaValidationMode = PROHIBIT_CHANGES): Seq[String] = { + val originSchema = CatalogV2Util.v2ColumnsToStructType(originCols) + val schema = CatalogV2Util.v2ColumnsToStructType(table.columns) + SchemaUtils.validateSchemaCompatibility(originSchema, schema, resolver, mode) } /** @@ -90,22 +77,25 @@ private[sql] object V2TableUtil extends SQLConfHelper { * * @param table the current table metadata * @param relation the relation with captured metadata columns + * @param mode validation mode that defines what changes are acceptable * @return validation errors, or empty sequence if valid */ - def validateCapturedMetadataColumns(table: Table, relation: DataSourceV2Relation): Seq[String] = { - validateCapturedMetadataColumns(table, extractMetadataColumns(relation)) + def validateCapturedMetadataColumns( + table: Table, + relation: DataSourceV2Relation, + mode: SchemaValidationMode): Seq[String] = { + validateCapturedMetadataColumns(table, extractMetadataColumns(relation), mode) } - // extracts original column info for all metadata attributes in relation + /** + * Extracts original column info for all metadata attributes in the relation. + * + * @param relation the relation with captured metadata columns + * @return metadata columns captured by the relation + */ def extractMetadataColumns(relation: DataSourceV2Relation): Seq[MetadataColumn] = { - val metaAttrs = relation.output.filter(_.isMetadataCol) - if (metaAttrs.nonEmpty) { - val metaCols = metadataColumns(relation.table) - val normalizedMetaAttrNames = metaAttrs.map(attr => normalize(attr.name)).toSet - metaCols.filter(col => normalizedMetaAttrNames.contains(normalize(col.name))) - } else { - Seq.empty - } + val metaAttrNames = relation.output.filter(_.isMetadataCol).map(_.name) + filter(metaAttrNames, metadataColumns(relation.table)) } /** @@ -117,56 +107,23 @@ private[sql] object V2TableUtil extends SQLConfHelper { * * @param table the current table metadata * @param originMetaCols the originally captured metadata columns + * @param mode validation mode that defines what changes are acceptable * @return validation errors, or empty sequence if valid */ def validateCapturedMetadataColumns( table: Table, - originMetaCols: Seq[MetadataColumn]): Seq[String] = { - val errors = mutable.ArrayBuffer[String]() - val metaCols = metadataColumns(table) - val metaColsByNormalizedName = indexMetadataColumns(metaCols) - - originMetaCols.foreach { originMetaCol => - val normalizedName = normalize(originMetaCol.name) - metaColsByNormalizedName.get(normalizedName) match { - case Some(metaCol) => - if (originMetaCol.dataType != metaCol.dataType || - originMetaCol.isNullable != metaCol.isNullable) { - val oldType = formatType(originMetaCol.dataType, originMetaCol.isNullable) - val newType = formatType(metaCol.dataType, metaCol.isNullable) - errors += s"`${originMetaCol.name}` type has changed from $oldType to $newType" - } - case None => - errors += s"${formatMetadataColumn(originMetaCol)} has been removed" - } - } - - errors.toSeq - } - - private def formatColumn(col: Column): String = { - s"`${col.name}` ${formatType(col.dataType, col.nullable)}" - } - - private def formatMetadataColumn(col: MetadataColumn): String = { - s"`${col.name}` ${formatType(col.dataType, col.isNullable)}" - } - - private def formatType(dataType: DataType, nullable: Boolean): String = { - if (nullable) dataType.sql else s"${dataType.sql} NOT NULL" - } - - private def indexColumns(cols: Seq[Column]): Map[String, Column] = { - index(cols)(_.name) + originMetaCols: Seq[MetadataColumn], + mode: SchemaValidationMode = PROHIBIT_CHANGES): Seq[String] = { + val originMetaColNames = originMetaCols.map(_.name) + val originMetaSchema = CatalogV2Util.toStructType(originMetaCols) + val metaCols = filter(originMetaColNames, metadataColumns(table)) + val metaSchema = CatalogV2Util.toStructType(metaCols) + SchemaUtils.validateSchemaCompatibility(originMetaSchema, metaSchema, resolver, mode) } - private def indexMetadataColumns(cols: Seq[MetadataColumn]): Map[String, MetadataColumn] = { - index(cols)(_.name) - } - - private def index[C](cols: Seq[C])(extractName: C => String): Map[String, C] = { - SchemaUtils.checkColumnNameDuplication(cols.map(extractName), conf.caseSensitiveAnalysis) - cols.map(col => normalize(extractName(col)) -> col).toMap + private def filter(colNames: Seq[String], cols: Seq[MetadataColumn]): Seq[MetadataColumn] = { + val normalizedColNames = colNames.map(normalize).toSet + cols.filter(col => normalizedColNames.contains(normalize(col.name))) } private def metadataColumns(table: Table): Seq[MetadataColumn] = table match { @@ -177,4 +134,6 @@ private[sql] object V2TableUtil extends SQLConfHelper { private def normalize(name: String): String = { if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT) } + + private def resolver: Resolver = conf.resolver } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 42c7541e4c214..76c4d518df6f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -20,15 +20,16 @@ package org.apache.spark.sql.util import java.util.Locale import scala.collection.immutable.Queue +import scala.collection.mutable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} -import org.apache.spark.sql.connector.catalog.CatalogV2Util -import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkSchemaUtils @@ -100,11 +101,6 @@ private[spark] object SchemaUtils { } } - def checkSchemaColumnNameDuplication(table: Table, resolver: Resolver): Unit = { - val schema = CatalogV2Util.v2ColumnsToStructType(table.columns) - checkSchemaColumnNameDuplication(schema, resolver) - } - /** * Checks if an input schema has duplicate column names. This throws an exception if the * duplication exists. @@ -396,4 +392,141 @@ private[spark] object SchemaUtils { case st: StringType => StringHelper.removeCollation(st) case _ => dt } + + /** + * Validates schema compatibility by recursively checking type and nullability changes. + * + * @param schema the schema to validate against + * @param otherSchema the other schema to check for compatibility + * @param resolver the resolver that controls whether the validation is case sensitive + * @param mode the validation mode that controls what changes are allowed + * @return sequence of error messages describing incompatibilities, empty if fully compatible + */ + def validateSchemaCompatibility( + schema: StructType, + otherSchema: StructType, + resolver: Resolver, + mode: SchemaValidationMode): Seq[String] = { + checkSchemaColumnNameDuplication(schema, resolver) + checkSchemaColumnNameDuplication(otherSchema, resolver) + val errors = mutable.ArrayBuffer[String]() + validateTypeCompatibility( + schema, + otherSchema, + nullable = false, + otherNullable = false, + colPath = Seq.empty, + resolver, + mode, + errors) + errors.toSeq + } + + private def validateTypeCompatibility( + dataType: DataType, + otherDataType: DataType, + nullable: Boolean, + otherNullable: Boolean, + colPath: Seq[String], + resolver: Resolver, + mode: SchemaValidationMode, + errors: mutable.ArrayBuffer[String]): Unit = { + if (nullable && !otherNullable) { + errors += s"${colPath.fullyQuoted} is no longer nullable" + } else if (!nullable && otherNullable) { + errors += s"${colPath.fullyQuoted} is nullable now" + } + + (dataType, otherDataType) match { + case (StructType(fields), StructType(otherFields)) => + val fieldsByName = index(fields, resolver) + val otherFieldsByName = index(otherFields, resolver) + + fieldsByName.foreach { case (normalizedName, field) => + otherFieldsByName.get(normalizedName) match { + case Some(otherField) => + validateTypeCompatibility( + field.dataType, + otherField.dataType, + field.nullable, + otherField.nullable, + colPath :+ field.name, + resolver, + mode, + errors) + case None => + errors += s"${formatField(colPath, field)} has been removed" + } + } + + if (mode == PROHIBIT_CHANGES) { + otherFieldsByName.foreach { case (normalizedName, otherField) => + if (!fieldsByName.contains(normalizedName)) { + errors += s"${formatField(colPath, otherField)} has been added" + } + } + } + + case (ArrayType(elem, containsNull), ArrayType(otherElem, otherContainsNull)) => + validateTypeCompatibility( + elem, + otherElem, + containsNull, + otherContainsNull, + colPath :+ "element", + resolver, + mode, + errors) + + case (MapType(keyType, valueType, valueContainsNull), + MapType(otherKeyType, otherValueType, otherValueContainsNull)) => + validateTypeCompatibility( + keyType, + otherKeyType, + nullable = false, + otherNullable = false, + colPath :+ "key", + resolver, + mode, + errors) + validateTypeCompatibility( + valueType, + otherValueType, + valueContainsNull, + otherValueContainsNull, + colPath :+ "value", + resolver, + mode, + errors) + + case _ if dataType != otherDataType => + errors += s"${colPath.fullyQuoted} type has changed " + + s"from ${dataType.sql} to ${otherDataType.sql}" + + case _ => + // OK + } + } + + private def formatField(colPath: Seq[String], field: StructField): String = { + val nameParts = colPath :+ field.name + val name = nameParts.fullyQuoted + val dataType = field.dataType.sql + if (field.nullable) s"$name $dataType" else s"$name $dataType NOT NULL" + } + + private def index(fields: Array[StructField], resolver: Resolver): Map[String, StructField] = { + if (isCaseSensitiveAnalysis(resolver)) { + fields.map(field => field.name -> field).toMap + } else { + fields.map(field => field.name.toLowerCase(Locale.ROOT) -> field).toMap + } + } +} + +private[spark] sealed trait SchemaValidationMode + +private[spark] object SchemaValidationMode { + case object PROHIBIT_CHANGES extends SchemaValidationMode + case object ALLOW_NEW_FIELDS extends SchemaValidationMode } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala index d484c8336603c..b4923b1169371 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES import org.apache.spark.util.ArrayImplicits.SparkArrayOps class V2TableUtilSuite extends SparkFunSuite { @@ -65,7 +66,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = validateCapturedColumns(table, originCols) assert(errors.size == 1) - assert(errors.head == "`id` type has changed from BIGINT to BIGINT NOT NULL") + assert(errors.head == "`id` is no longer nullable") } test("validateCapturedColumns - column nullability changed to nullable") { @@ -79,7 +80,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = validateCapturedColumns(table, originCols) assert(errors.size == 1) - assert(errors.head == "`id` type has changed from BIGINT NOT NULL to BIGINT") + assert(errors.head == "`id` is nullable now") } test("validateCapturedColumns - column removed") { @@ -187,9 +188,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = V2TableUtil.validateCapturedColumns(table, originCols.toSeq) assert(errors.size == 1) - assert(errors.head.contains( - "`address` type has changed from STRUCT to " + - "STRUCT")) + assert(errors.head.contains("`address`.`city` type has changed from STRING to INT")) } test("validateCapturedMetadataColumns - no changes") { @@ -214,7 +213,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) assert(errors.size == 1) - assert(errors.head == "`index` type has changed from INT NOT NULL to STRING NOT NULL") + assert(errors.head == "`index` type has changed from INT to STRING") } test("validateCapturedMetadataColumns - nullability changed to nullable") { @@ -226,7 +225,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) assert(errors.size == 1) - assert(errors.head == "`index` type has changed from INT NOT NULL to INT") + assert(errors.head == "`index` is nullable now") } test("validateCapturedMetadataColumns - nullability changed to not null") { @@ -236,7 +235,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) assert(errors.size == 1) - assert(errors.head == "`index` type has changed from INT to INT NOT NULL") + assert(errors.head == "`index` is no longer nullable") } test("validateCapturedMetadataColumns - column removed") { @@ -332,9 +331,7 @@ class V2TableUtilSuite extends SparkFunSuite { val errors = V2TableUtil.validateCapturedMetadataColumns(table, originMetaCols) assert(errors.size == 1) - assert(errors.head.contains( - "`_partition` type has changed from STRUCT NOT NULL to " + - "STRUCT NOT NULL")) + assert(errors.head.contains("`_partition`.`bucket` type has changed from INT to STRING")) } test("validateCapturedMetadataColumns - with DataSourceV2Relation") { @@ -362,7 +359,10 @@ class V2TableUtilSuite extends SparkFunSuite { metaCol("index", IntegerType, nullable = false)) val currentTable = TestTableWithMetadataSupport("test", dataCols, currentMetaCols) - val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation) + val errors = V2TableUtil.validateCapturedMetadataColumns( + currentTable, + relation, + mode = PROHIBIT_CHANGES) assert(errors.size == 1) assert(errors.head.contains("`_partition` type has changed")) } @@ -384,10 +384,151 @@ class V2TableUtilSuite extends SparkFunSuite { val currentTable = TestTable("test", dataCols) - val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation) + val errors = V2TableUtil.validateCapturedMetadataColumns( + currentTable, + relation, + mode = PROHIBIT_CHANGES) assert(errors.isEmpty) } + test("validateCapturedColumns - array element type changed") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", ArrayType(IntegerType, containsNull = true), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", ArrayType(LongType, containsNull = true), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`element` type has changed from INT to BIGINT") + } + + test("validateCapturedColumns - array containsNull changed to false") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", ArrayType(IntegerType, containsNull = true), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", ArrayType(IntegerType, containsNull = false), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`element` is no longer nullable") + } + + test("validateCapturedColumns - array containsNull changed to true") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", ArrayType(IntegerType, containsNull = false), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", ArrayType(IntegerType, containsNull = true), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`element` is nullable now") + } + + test("validateCapturedColumns - map key type changed") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(IntegerType, StringType, valueContainsNull = true), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(LongType, StringType, valueContainsNull = true), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`key` type has changed from INT to BIGINT") + } + + test("validateCapturedColumns - map value type changed") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(StringType, IntegerType, valueContainsNull = true), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(StringType, LongType, valueContainsNull = true), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`value` type has changed from INT to BIGINT") + } + + test("validateCapturedColumns - map valueContainsNull changed to false") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(StringType, IntegerType, valueContainsNull = true), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(StringType, IntegerType, valueContainsNull = false), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`value` is no longer nullable") + } + + test("validateCapturedColumns - map valueContainsNull changed to true") { + val originCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(StringType, IntegerType, valueContainsNull = false), nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = true), + col("data", MapType(StringType, IntegerType, valueContainsNull = true), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`data`.`value` is nullable now") + } + + test("validateCapturedColumns - nested array in struct element type changed") { + val originStructType = StructType(Seq( + StructField("name", StringType), + StructField("scores", ArrayType(IntegerType, containsNull = true)))) + val originCols = Array( + col("id", LongType, nullable = true), + col("person", originStructType, nullable = true)) + val currentStructType = StructType(Seq( + StructField("name", StringType), + StructField("scores", ArrayType(LongType, containsNull = true)))) + val currentCols = Array( + col("id", LongType, nullable = true), + col("person", currentStructType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`person`.`scores`.`element` type has changed from INT to BIGINT") + } + + test("validateCapturedColumns - nested map in struct value type changed") { + val originStructType = StructType(Seq( + StructField("name", StringType), + StructField("attrs", MapType(StringType, IntegerType, valueContainsNull = true)))) + val originCols = Array( + col("id", LongType, nullable = true), + col("person", originStructType, nullable = true)) + val currentStructType = StructType(Seq( + StructField("name", StringType), + StructField("attrs", MapType(StringType, LongType, valueContainsNull = true)))) + val currentCols = Array( + col("id", LongType, nullable = true), + col("person", currentStructType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols) + assert(errors.size == 1) + assert(errors.head == "`person`.`attrs`.`value` type has changed from INT to BIGINT") + } + // simple table without metadata column support private case class TestTable( override val name: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala index 7071e6fd4ff51..4652c9441d416 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala @@ -22,11 +22,14 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.AsOfVersion -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog, V2TableUtil} import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.util.SchemaValidationMode +import org.apache.spark.sql.util.SchemaValidationMode.ALLOW_NEW_FIELDS +import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { /** @@ -55,12 +58,15 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { * * This method reloads table metadata from the catalog and validates: * - Table identity: Ensures table ID has not changed - * - Data columns: Verifies captured columns match the current schema + * - Data columns: Verifies captured columns align with the current schema * - Metadata columns: Checks metadata column consistency * * Tables with time travel specifications are skipped as they reference a specific point * in time and don't have to be refreshed. * + * Schema validation mode depends on the underlying plan. Commands, for instance, + * prohibit any schema changes while queries permit adding columns. + * * @param spark the currently active Spark session * @param plan the logical plan to refresh * @param versionedOnly indicates whether to refresh only versioned tables @@ -70,6 +76,31 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { spark: SparkSession, plan: LogicalPlan, versionedOnly: Boolean = false): LogicalPlan = { + refresh(spark, plan, versionedOnly, determineSchemaValidationMode(plan)) + } + + /** + * Refreshes table metadata for tables in the plan. + * + * This method reloads table metadata from the catalog and validates: + * - Table identity: Ensures table ID has not changed + * - Data columns: Verifies captured columns align with the current schema + * - Metadata columns: Checks metadata column consistency + * + * Tables with time travel specifications are skipped as they reference a specific point + * in time and don't have to be refreshed. + * + * @param spark the currently active Spark session + * @param plan the logical plan to refresh + * @param versionedOnly indicates whether to refresh only versioned tables + * @param schemaValidationMode schema validation mode to use + * @return plan with refreshed table metadata + */ + def refresh( + spark: SparkSession, + plan: LogicalPlan, + versionedOnly: Boolean, + schemaValidationMode: SchemaValidationMode): LogicalPlan = { val currentTables = mutable.HashMap.empty[(TableCatalog, Identifier), Table] plan transform { case r @ ExtractV2CatalogAndIdentifier(catalog, ident) @@ -86,8 +117,8 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { } }) validateTableIdentity(currentTable, r) - validateDataColumns(currentTable, r) - validateMetadataColumns(currentTable, r) + validateDataColumns(currentTable, r, schemaValidationMode) + validateMetadataColumns(currentTable, r, schemaValidationMode) r.copy(table = currentTable) } } @@ -100,6 +131,15 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { CatalogV2Util.lookupCachedRelation(spark.sharedState.relationCache, catalog, ident, table, conf) } + // it is not safe to allow any schema changes in commands (e.g. CTAS, RTAS, MERGE) + private def determineSchemaValidationMode(plan: LogicalPlan): SchemaValidationMode = { + if (containsCommand(plan)) PROHIBIT_CHANGES else ALLOW_NEW_FIELDS + } + + private def containsCommand(plan: LogicalPlan): Boolean = { + plan.find(_.isInstanceOf[Command]).isDefined + } + private def validateTableIdentity(currentTable: Table, relation: DataSourceV2Relation): Unit = { if (relation.table.id != null && relation.table.id != currentTable.id) { throw QueryCompilationErrors.tableIdChangedAfterAnalysis( @@ -109,15 +149,21 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { } } - private def validateDataColumns(currentTable: Table, relation: DataSourceV2Relation): Unit = { - val errors = V2TableUtil.validateCapturedColumns(currentTable, relation) + private def validateDataColumns( + currentTable: Table, + relation: DataSourceV2Relation, + mode: SchemaValidationMode): Unit = { + val errors = V2TableUtil.validateCapturedColumns(currentTable, relation, mode) if (errors.nonEmpty) { throw QueryCompilationErrors.columnsChangedAfterAnalysis(relation.name, errors) } } - private def validateMetadataColumns(currentTable: Table, relation: DataSourceV2Relation): Unit = { - val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation) + private def validateMetadataColumns( + currentTable: Table, + relation: DataSourceV2Relation, + mode: SchemaValidationMode): Unit = { + val errors = V2TableUtil.validateCapturedMetadataColumns(currentTable, relation, mode) if (errors.nonEmpty) { throw QueryCompilationErrors.metadataColumnsChangedAfterAnalysis(relation.name, errors) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 9e5e45e984eb5..6c574be91ebf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.joins.BaseJoinExec import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric, SQLMetrics} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES import org.apache.spark.util.{LongAccumulator, Utils} import org.apache.spark.util.ArrayImplicits._ @@ -171,7 +172,11 @@ case class ReplaceTableAsSelectExec( // // RTAS must refresh and pin versions in query to read from original table versions instead of // newly created empty table that is meant to serve as target for append/overwrite - val refreshedQuery = V2TableRefreshUtil.refresh(session, query, versionedOnly = true) + val refreshedQuery = V2TableRefreshUtil.refresh( + session, + query, + versionedOnly = true, + schemaValidationMode = PROHIBIT_CHANGES) val pinnedQuery = V2TableRefreshUtil.pinVersions(refreshedQuery) if (catalog.tableExists(ident)) { invalidateCache(catalog, ident) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index e4b45ec66462a..7f4ad348f2446 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1124,7 +1124,7 @@ class DataSourceV2DataFrameSuite } } - test("SPARK-54157: detect column addition after DataFrame analysis") { + test("SPARK-54157: allow column addition after DataFrame analysis") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id INT, data STRING) USING foo") @@ -1137,22 +1137,15 @@ class DataSourceV2DataFrameSuite sql(s"ALTER TABLE $t ADD COLUMN new_col1 INT") sql(s"ALTER TABLE $t ADD COLUMN new_col2 INT") - // execution should fail with column mismatch - checkError( - exception = intercept[AnalysisException] { df.collect() }, - condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", - parameters = Map( - "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "errors" -> - """- `new_col1` INT has been added - |- `new_col2` INT has been added""".stripMargin)) + // execution should succeed as column additions are allowed + checkAnswer(df, Seq(Row(1, "a"))) } } test("SPARK-54157: detect multiple change types after DataFrame analysis") { val t = "testcat.ns1.ns2.tbl" withTable(t) { - sql(s"CREATE TABLE $t (col1 INT, col2 STRING, col3 BOOLEAN, col4 STRING) USING foo") + sql(s"CREATE TABLE $t (col1 INT, col2 STRING, col3 BOOLEAN NOT NULL, col4 STRING) USING foo") sql(s"INSERT INTO $t VALUES (1, 'a', true, 'x')") // create DataFrame and trigger analysis @@ -1160,7 +1153,7 @@ class DataSourceV2DataFrameSuite // make multiple changes in table sql(s"ALTER TABLE $t DROP COLUMN col4") - sql(s"ALTER TABLE $t ADD COLUMN col5 INT") + sql(s"ALTER TABLE $t ALTER COLUMN col3 DROP NOT NULL") // execution should fail with column mismatch checkError( @@ -1169,8 +1162,8 @@ class DataSourceV2DataFrameSuite parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "errors" -> - """- `col4` STRING has been removed - |- `col5` INT has been added""".stripMargin)) + """- `col3` is nullable now + |- `col4` STRING has been removed""".stripMargin)) } } @@ -1200,7 +1193,7 @@ class DataSourceV2DataFrameSuite } } - test("SPARK-54157: detect nested struct field changes after DataFrame analysis") { + test("SPARK-54157: detect incompatible nested struct field changes after DataFrame analysis") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id INT, person STRUCT) USING foo") @@ -1209,8 +1202,8 @@ class DataSourceV2DataFrameSuite // create DataFrame and trigger analysis val df = spark.table(t) - // add nested field to struct column - sql(s"ALTER TABLE $t ADD COLUMN person.city STRING") + // remove nested field from struct column + sql(s"ALTER TABLE $t DROP COLUMN person.age") // execution should fail with column mismatch checkError( @@ -1218,13 +1211,11 @@ class DataSourceV2DataFrameSuite condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "errors" -> - ("- `person` type has changed from STRUCT " + - "to STRUCT"))) + "errors" -> "- `person`.`age` INT has been removed")) } } - test("SPARK-54157: detect schema changes in join with same table") { + test("SPARK-54157: allow compatible schema changes in join with same table") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id INT, name STRING, value INT) USING foo") @@ -1260,8 +1251,63 @@ class DataSourceV2DataFrameSuite Row(3, "c", 30, null), Row(4, "d", 40, "x"))) - // join between df1 and df3 should fail as refreshing versions is not - // sufficient because df1 was resolved with old schema + // join between df1 and df3 is allowed as schema changes are compatible with df1 + // Spark will refresh versions in joined DataFrame before execution + checkAnswer(df1.join(df3, df1("id") === df3("id")), Seq( + Row(1, "a", 10, 1, "a", 10, null), + Row(2, "b", 20, 2, "b", 20, null), + Row(3, "c", 30, 3, "c", 30, null), + Row(4, "d", 40, 4, "d", 40, "x"))) + + // DataFrame execution before joins must have pinned used versions + // subsequent version refreshes must not be visible in original DataFrames + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) + checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + checkAnswer(df3, Seq( + Row(1, "a", 10, null), + Row(2, "b", 20, null), + Row(3, "c", 30, null), + Row(4, "d", 40, "x"))) + } + } + + test("SPARK-54157: prohibit incompatible schema changes in join with same table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, name STRING, value INT) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a', 10), (2, 'b', 20)") + + // create first DataFrame + val df1 = spark.table(t) + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) + + // insert more data + sql(s"INSERT INTO $t VALUES (3, 'c', 30)") + + // create second DataFrame with new data + val df2 = spark.table(t) + checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) + + // it should be valid to join df1 and df2 + // Spark will refresh versions in joined DataFrame before execution + assert(df1.join(df2, df1("id") === df2("id")).count() == 3) + + // df1 has been executed that must have pinned the version + checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) + + // remove column and insert more data + sql(s"ALTER TABLE $t DROP COLUMN value") + sql(s"INSERT INTO $t VALUES (4, 'd')") + + // create third DataFrame with new data and schema + val df3 = spark.table(t) + checkAnswer(df3, Seq( + Row(1, "a"), + Row(2, "b"), + Row(3, "c"), + Row(4, "d"))) + + // join between df1 and df3 should fail due to incompatible schema changes checkError( exception = intercept[AnalysisException] { df1.join(df3, df1("id") === df3("id")).collect() @@ -1269,17 +1315,17 @@ class DataSourceV2DataFrameSuite condition = "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS.COLUMNS_MISMATCH", parameters = Map( "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "errors" -> "- `extra` STRING has been added")) + "errors" -> "- `value` INT has been removed")) // DataFrame execution before joins must have pinned used versions // subsequent version refreshes must not be visible in original DataFrames checkAnswer(df1, Seq(Row(1, "a", 10), Row(2, "b", 20))) checkAnswer(df2, Seq(Row(1, "a", 10), Row(2, "b", 20), Row(3, "c", 30))) checkAnswer(df3, Seq( - Row(1, "a", 10, null), - Row(2, "b", 20, null), - Row(3, "c", 30, null), - Row(4, "d", 40, "x"))) + Row(1, "a"), + Row(2, "b"), + Row(3, "c"), + Row(4, "d"))) } } @@ -1447,7 +1493,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "- `data` type has changed from STRING NOT NULL to STRING")) + "errors" -> "- `data` is nullable now")) } } @@ -1707,6 +1753,33 @@ class DataSourceV2DataFrameSuite } } + test("SPARK-54444: any schema changes after analysis are prohibited in commands") { + val s = "testcat.ns1.s" + val t = "testcat.ns1.t" + withTable(s, t) { + sql(s"CREATE TABLE $s (id bigint, data string) USING foo") + sql(s"INSERT INTO $s VALUES (1, 'a'), (2, 'b')") + + // create source DataFrame without executing it + val sourceDF = spark.table(s) + + // derive another DataFrame from pre-analyzed source + val filteredSourceDF = sourceDF.filter("id < 10") + + // add column + sql(s"ALTER TABLE $s ADD COLUMN dep STRING") + + // insert more data into source table + sql(s"INSERT INTO $s VALUES (3, 'c', 'finance')") + + // CTAS should fail as commands must operate on current schema + val e = intercept[AnalysisException] { + filteredSourceDF.writeTo(t).createOrReplace() + } + assert(e.message.contains("incompatible changes to table `testcat`.`ns1`.`s`")) + } + } + private def pinTable(catalogName: String, ident: Identifier, version: String): Unit = { catalog(catalogName) match { case inMemory: BasicInMemoryTableCatalog => inMemory.pinTable(ident, version) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala index 9d666b2f0f2f4..e1c574ec7ba65 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.Row import org.apache.spark.sql.classic.MergeIntoWriter import org.apache.spark.sql.connector.catalog.Column @@ -1023,4 +1024,53 @@ class MergeIntoDataFrameSuite extends RowLevelOperationSuiteBase { Row(4, 401, "finance"))) // insert } } + + test("SPARK-54444: any schema changes after analysis are prohibited") { + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin) + + // create source table + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("salary", IntegerType), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + sql(s"INSERT INTO $sourceTable VALUES (1, 101, 'support'), (3, 301, 'support')") + + // create source DataFrame without executing it + val sourceDF = spark.table(sourceTable) + + // derive another DataFrame from pre-analyzed source + val filteredSourceDF = sourceDF.filter("pk < 10") + + // add column + sql(s"ALTER TABLE $sourceTable ADD COLUMN new_col INT") + + // insert more data into source table + sql(s"INSERT INTO $sourceTable VALUES (4, 401, 'finance', 4)") + + // merge should fail as commands must operate on current schema + val e = intercept[AnalysisException] { + filteredSourceDF + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .withSchemaEvolution() + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .merge() + } + assert(e.message.contains("incompatible changes to table `cat`.`ns1`.`source_table`")) + } + } } From 7f11fc5055fc37dcc18d578b154f951fc7678d75 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 25 Nov 2025 13:09:41 -0800 Subject: [PATCH 218/400] [SPARK-54500][PYTHON][TESTS] Restore `pythonUDF.arrow.concurrency.level` in ArrowPythonUDFLegacyTests ### What changes were proposed in this pull request? Restore the configuration of `spark.sql.execution.pythonUDF.arrow.concurrency.level` in ArrowPythonUDFLegacyTests. ### Why are the changes needed? The conf setting was removed in a test refactor. Restore to the prior setting from https://issues.apache.org/jira/browse/SPARK-50446 ### Does this PR introduce _any_ user-facing change? No, test only ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #53210 from asl3/asl3/arrpytest-concurrency. Authored-by: Amanda Liu Signed-off-by: Dongjoon Hyun (cherry picked from commit 3095eee5562678eb2ceb8d92d6c6d5e83a2c20bb) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/arrow/test_arrow_python_udf.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 90e05caf21800..c315151d4d759 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -508,11 +508,13 @@ class ArrowPythonUDFLegacyTests(ArrowPythonUDFLegacyTestsMixin, ReusedSQLTestCas @classmethod def setUpClass(cls): super(ArrowPythonUDFLegacyTests, cls).setUpClass() + cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.concurrency.level", "4") cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "true") @classmethod def tearDownClass(cls): try: + cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.concurrency.level") cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.enabled") finally: super(ArrowPythonUDFLegacyTests, cls).tearDownClass() From 9b1967c903658aed31bc81463a8e55775c983c16 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 25 Nov 2025 13:14:37 -0800 Subject: [PATCH 219/400] [SPARK-54504][SQL] Fix version refresh for DSv2 tables with subqueries ### What changes were proposed in this pull request? This PR fixes version refresh for DSv2 tables with subqueries to be released in 4.1. ### Why are the changes needed? These changes are needed for correctness. See tests below. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR comes with a test that was previously failing. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53214 from aokolnychyi/spark-54505. Lead-authored-by: Anton Okolnychyi Co-authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun (cherry picked from commit 754eda2c9c28bb1bfc4ea8ef0a5501be03bda65d) Signed-off-by: Dongjoon Hyun --- .../datasources/v2/V2TableRefreshUtil.scala | 2 +- .../DataSourceV2DataFrameSuite.scala | 28 +++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala index 4652c9441d416..945ab122d54ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2TableRefreshUtil.scala @@ -102,7 +102,7 @@ private[sql] object V2TableRefreshUtil extends SQLConfHelper with Logging { versionedOnly: Boolean, schemaValidationMode: SchemaValidationMode): LogicalPlan = { val currentTables = mutable.HashMap.empty[(TableCatalog, Identifier), Table] - plan transform { + plan transformWithSubqueries { case r @ ExtractV2CatalogAndIdentifier(catalog, ident) if (r.isVersioned || !versionedOnly) && r.timeTravelSpec.isEmpty => val currentTable = currentTables.getOrElseUpdate((catalog, ident), { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 7f4ad348f2446..d8d68f576e4ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1753,6 +1753,34 @@ class DataSourceV2DataFrameSuite } } + test("SPARK-54504: self-subquery refreshes both table references before execution") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id INT, value INT) USING foo") + sql(s"INSERT INTO $t VALUES (1, 10), (2, 20)") + + // create DataFrame with self-subquery without executing + val df = spark.sql( + s""" + |SELECT t1.id, t1.value, t2.value as other_value + |FROM $t t1 + |JOIN ( + | SELECT id, value FROM $t + | WHERE id IN (SELECT id FROM $t WHERE value > 5) + |) t2 ON t1.id = t2.id + |""".stripMargin) + + // insert more data into base table + sql(s"INSERT INTO $t VALUES (3, 30)") + + // all three table references should be refreshed to see new data + checkAnswer(df, Seq( + Row(1, 10, 10), + Row(2, 20, 20), + Row(3, 30, 30))) + } + } + test("SPARK-54444: any schema changes after analysis are prohibited in commands") { val s = "testcat.ns1.s" val t = "testcat.ns1.t" From 4c03da6f5aab82b5786a7b6acc10f468b098f367 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 25 Nov 2025 13:40:25 -0800 Subject: [PATCH 220/400] [SPARK-54508][PYTHON] Fix `spark-pipelines` to resolve `spec` file path more robustly ### What changes were proposed in this pull request? Fixes a bug that causes executing `spark-pipelines` with a spec file in a subdirectory to fail: ``` spark-pipelines run --spec subdir/spark-pipelines.yml ``` results in ``` pyspark.errors.exceptions.connect.AnalysisException: [RUN_EMPTY_PIPELINE] Pipelines are expected to have at least one non-temporary dataset defined (tables, persisted views) but no non-temporary datasets were found in your pipeline. ``` ### Why are the changes needed? Fix a bug. ### Does this PR introduce _any_ user-facing change? Fixes a user-facing bug ### How was this patch tested? Ran ``` spark-pipelines run --spec subdir/spark-pipelines.yml ``` Observed it fail before the change and succeed after. With our current Pipelines CLI unit testing setup, there isn't a straightforward way to write a test for this, but I'm investigating whether we can augment it. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53219 from sryza/cli-relative-path. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit c61d40c88f7bb50e81b7fb98852a96b2e6296b59) Signed-off-by: Dongjoon Hyun --- python/pyspark/pipelines/cli.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pipelines/cli.py b/python/pyspark/pipelines/cli.py index 3e1c532897245..8c70c83311201 100644 --- a/python/pyspark/pipelines/cli.py +++ b/python/pyspark/pipelines/cli.py @@ -228,7 +228,8 @@ def register_definitions( - Import Python files matching the glob patterns in the spec. - Register SQL files matching the glob patterns in the spec. """ - path = spec_path.parent + path = spec_path.parent.resolve() + with change_dir(path): with graph_element_registration_context(registry): log_with_curr_timestamp(f"Loading definitions. Root directory: '{path}'.") @@ -260,7 +261,7 @@ def register_definitions( log_with_curr_timestamp(f"Registering SQL file {file}...") with file.open("r") as f: sql = f.read() - file_path_relative_to_spec = file.relative_to(spec_path.parent) + file_path_relative_to_spec = file.relative_to(path) registry.register_sql(sql, file_path_relative_to_spec) else: raise PySparkException( From 5f6eaa1237a98e78ce4df12ac9f16c96df4d4dae Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 25 Nov 2025 13:50:46 -0800 Subject: [PATCH 221/400] [SPARK-54503][PYTHON][DOCS] Include `pyspark.pipelines` in API reference ### What changes were proposed in this pull request? Includes the pyspark.pipelines package in the API reference. ### Why are the changes needed? To document pyspark.pipelines functionality. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #53213 from sryza/api-reference. Authored-by: Sandy Ryza Signed-off-by: Dongjoon Hyun (cherry picked from commit 64c9d89dddcdef5c798f3ae5ff5e500eadbdf7d7) Signed-off-by: Dongjoon Hyun --- python/docs/source/index.rst | 11 ++++++- python/docs/source/reference/index.rst | 1 + .../source/reference/pyspark.pipelines.rst | 33 +++++++++++++++++++ python/pyspark/pipelines/api.py | 4 +-- 4 files changed, 46 insertions(+), 3 deletions(-) create mode 100644 python/docs/source/reference/pyspark.pipelines.rst diff --git a/python/docs/source/index.rst b/python/docs/source/index.rst index c8d3fe62bf3b3..b412f2746bdd2 100644 --- a/python/docs/source/index.rst +++ b/python/docs/source/index.rst @@ -34,7 +34,7 @@ PySpark combines Python's learnability and ease of use with the power of Apache to enable processing and analysis of data at any size for everyone familiar with Python. PySpark supports all of Spark's features such as Spark SQL, -DataFrames, Structured Streaming, Machine Learning (MLlib) and Spark Core. +DataFrames, Structured Streaming, Machine Learning (MLlib), Pipelines and Spark Core. .. list-table:: :widths: 10 80 10 @@ -151,6 +151,15 @@ learning pipelines. - `Machine Learning Library (MLlib) Programming Guide `_ - :ref:`Machine Learning (MLlib) API Reference` +**Declarative Pipelines** + +Spark Declarative Pipelines (SDP) is a declarative framework for building reliable, +maintainable, and testable data pipelines on Spark. SDP simplifies ETL development by allowing +you to focus on the transformations you want to apply to your data, rather than the mechanics +of pipeline execution. + +- :ref:`Pipelines API Reference` + **Spark Core and RDDs** Spark Core is the underlying general execution engine for the Spark platform that all diff --git a/python/docs/source/reference/index.rst b/python/docs/source/reference/index.rst index 0068c0b2322e2..11c180c4825e7 100644 --- a/python/docs/source/reference/index.rst +++ b/python/docs/source/reference/index.rst @@ -36,6 +36,7 @@ This page lists an overview of all public PySpark modules, classes, functions an pyspark.streaming pyspark.mllib pyspark + pyspark.pipelines pyspark.resource pyspark.errors pyspark.logger diff --git a/python/docs/source/reference/pyspark.pipelines.rst b/python/docs/source/reference/pyspark.pipelines.rst new file mode 100644 index 0000000000000..e3f7384334442 --- /dev/null +++ b/python/docs/source/reference/pyspark.pipelines.rst @@ -0,0 +1,33 @@ +.. 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. + + +================== +PySpark Pipelines +================== + +.. currentmodule:: pyspark.pipelines + +.. autosummary:: + :toctree: api/ + + materialized_view + table + temporary_view + create_streaming_table + append_flow + create_sink diff --git a/python/pyspark/pipelines/api.py b/python/pyspark/pipelines/api.py index 7f8ba4297f3c7..f109841d657b8 100644 --- a/python/pyspark/pipelines/api.py +++ b/python/pyspark/pipelines/api.py @@ -421,7 +421,7 @@ def create_streaming_table( Creates a table that can be targeted by append flows. Example: - create_streaming_table("target") + create_streaming_table("target") :param name: The name of the table. :param comment: Description of the table. @@ -429,7 +429,7 @@ def create_streaming_table( property values. These properties will be set on the table. :param partition_cols: A list containing the column names of the partition columns. :param cluster_by: A list containing the column names of the cluster columns. - :param schema Explicit Spark SQL schema to materialize this table with. Supports either a \ + :param schema: Explicit Spark SQL schema to materialize this table with. Supports either a \ Pyspark StructType or a SQL DDL string, such as "a INT, b STRING". :param format: The format of the table, e.g. "parquet". """ From 3baf3369f1a771021d8b2d083db8fb7c85209482 Mon Sep 17 00:00:00 2001 From: David Milicevic Date: Tue, 25 Nov 2025 13:55:33 -0800 Subject: [PATCH 222/400] [SPARK-54499][SQL][4.1] Re-enable `spark.sql.scripting.enabled` by default for Spark 4.1.0 ### What changes were proposed in this pull request? This PR aims to re-enable SQL Scripting in Spark 4.1, ahead of its RC. ### Why are the changes needed? There was a previous misunderstanding/miscommunication, that I did not notice in time. The feature was disabled because it was thought that it is not ready, due to unclosed JIRA items. However, it was just a case of not 100% up-to-date items. I have cleaned up and classified remaining work items (children of [SPARK-48338](https://issues.apache.org/jira/browse/SPARK-48338)) into 4 categories: - M0 - basic support - M1 - features and changes required to enable SQL Scripting by default - M2 - follow-up improvements and additional functionalities that are non-fundamental and should not block M1 - M3 - potential improvements for the future, need investigation M0 and M1 are done, meaning the feature is stable, useful and also ready to be used. M2 will improve some of the aspects of using it by introducing a newer and more user-friendly statements, like SIGNAL/RESIGNAL and GET DIAGNOSTICS, as well as do some very minor optimizations. However, without those, feature is still ready to be used as is. The only missing aspect is documentation, for which we have a PR (https://github.com/apache/spark/pull/50592) but it was closed. I'll work on re-submitting that one. ### Does this PR introduce _any_ user-facing change? I am not sure 100% how this classifies - the feature was already present in 4.0, but was not enabled by default. Anyways, the feature is completely orthogonal to all other standalone statement execution paths, so it is just adding to functionality of Spark. There is no difference in behavior for any standalone SQL statement compared to when SQL Scripting is turned off. ### How was this patch tested? SQL Scripting has a thorough support in tests. CI ensures the feature does not affect any other. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53154 from davidm-db/davidm-db/enable-sql-scripting-4.1. Authored-by: David Milicevic Signed-off-by: Dongjoon Hyun --- docs/sql-ref-scripting.md | 4 ---- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/sql-ref-scripting.md b/docs/sql-ref-scripting.md index a8bccb471d563..e9407cff2301f 100644 --- a/docs/sql-ref-scripting.md +++ b/docs/sql-ref-scripting.md @@ -32,10 +32,6 @@ This is followed by the compound statement body, which consists of: - The [EXECUTE IMMEDIATE](sql-ref-syntax-aux-exec-imm.html) statement. - Nested compound statements, which provide nested scopes for variables, conditions, and condition handlers. -## Enablement - -To enable SQL Scripting, set the `spark.sql.scripting.enabled` flag to `true`. - ## Passing data between the invoker and the compound statement There are two ways to pass data to and from a SQL script: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 18cd916cb5d24..4b82966b2b6d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4336,7 +4336,7 @@ object SQLConf { "flow and error handling.") .version("4.0.0") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val SQL_SCRIPTING_CONTINUE_HANDLER_ENABLED = buildConf("spark.sql.scripting.continueHandlerEnabled") From 807a6ef07b3d34398a4d9be7cae917828f828efa Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 26 Nov 2025 08:35:16 +0900 Subject: [PATCH 223/400] [SPARK-54482][PYTHON][CONNECT][TESTS] Reeanble `test_apply_in_pandas_returning_column_names_*` for Spark Connect 4.0 client <> master server ### What changes were proposed in this pull request? Fix tests ### Why are the changes needed? to make CI happy ### Does this PR introduce _any_ user-facing change? No, test-only ### How was this patch tested? will monitor the scheduled jobs ### Was this patch authored or co-authored using generative AI tooling? No Closes #53209 from zhengruifeng/test_aply_in_pd. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon (cherry picked from commit 27a4849834406a5bbfb0a0b11ea8b725936baef6) Signed-off-by: Hyukjin Kwon --- .../tests/pandas/test_pandas_grouped_map.py | 34 ++++--------------- 1 file changed, 7 insertions(+), 27 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index f8c6469338090..45b18e9414eb1 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -278,37 +278,20 @@ def check_apply_in_pandas_not_returning_pandas_dataframe(self): ): self._test_apply_in_pandas(lambda key, pdf: key) - @staticmethod - def stats_with_column_names(key, pdf): - # order of column can be different to applyInPandas schema when column names are given - return pd.DataFrame([(pdf.v.mean(),) + key], columns=["mean", "id"]) - - @staticmethod - def stats_with_no_column_names(key, pdf): - # columns must be in order of applyInPandas schema when no columns given - return pd.DataFrame([key + (pdf.v.mean(),)]) - - @unittest.skipIf( - os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" - ) def test_apply_in_pandas_returning_column_names(self): - self._test_apply_in_pandas(ApplyInPandasTestsMixin.stats_with_column_names) + self._test_apply_in_pandas( + lambda key, pdf: pd.DataFrame([(pdf.v.mean(),) + key], columns=["mean", "id"]) + ) - @unittest.skipIf( - os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" - ) def test_apply_in_pandas_returning_no_column_names(self): - self._test_apply_in_pandas(ApplyInPandasTestsMixin.stats_with_no_column_names) + self._test_apply_in_pandas(lambda key, pdf: pd.DataFrame([key + (pdf.v.mean(),)])) - @unittest.skipIf( - os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" - ) def test_apply_in_pandas_returning_column_names_sometimes(self): def stats(key, pdf): if key[0] % 2: - return ApplyInPandasTestsMixin.stats_with_column_names(key, pdf) + return pd.DataFrame([(pdf.v.mean(),) + key], columns=["mean", "id"]) else: - return ApplyInPandasTestsMixin.stats_with_no_column_names(key, pdf) + return pd.DataFrame([key + (pdf.v.mean(),)]) self._test_apply_in_pandas(stats) @@ -886,7 +869,7 @@ def _test_apply_in_pandas_returning_empty_dataframe(self, empty_df): def stats(key, pdf): if key[0] % 2 == 0: - return ApplyInPandasTestsMixin.stats_with_no_column_names(key, pdf) + return pd.DataFrame([key + (pdf.v.mean(),)]) return empty_df result = ( @@ -903,9 +886,6 @@ def stats(key, pdf): for row in result: self.assertEqual(24.5, row[1]) - @unittest.skipIf( - os.environ.get("SPARK_SKIP_CONNECT_COMPAT_TESTS") == "1", "SPARK-54482: To be reenabled" - ) def _test_apply_in_pandas_returning_empty_dataframe_error(self, empty_df, error): with self.quiet(): with self.assertRaisesRegex(PythonException, error): From 9e4e780a582017feb13dfcc62a68f09a6fe5bc1a Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Tue, 25 Nov 2025 22:06:52 -0800 Subject: [PATCH 224/400] [SPARK-54496][SQL] Fix Merge Into Schema Evolution for Dataframe API ### What changes were proposed in this pull request? Some fixes to allow the Dataframe Merge API to support schema evolution. The DataFrame API is here: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/classic/MergeIntoWriter.scala#L7 The fixes are described inline. ### Why are the changes needed? The Dataframe Merge API is broken for schema evolution mode without these fixes. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add unit tests. Will try to refactor later to combine test re-use. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53207 from szehon-ho/merge_schema_evolution_bug. Authored-by: Szehon Ho Signed-off-by: Dongjoon Hyun (cherry picked from commit 9feb1b2c02202fcf04c2dc9a4f44fcd6c63cdeb8) Signed-off-by: Dongjoon Hyun --- .../ResolveMergeIntoSchemaEvolution.scala | 15 +- .../catalyst/plans/logical/v2Commands.scala | 30 +- .../connector/MergeIntoTableSuiteBase.scala | 999 +++++++++++++++++- 3 files changed, 1028 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala index ea0883f7928f2..bbb8e7852b2c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsRowLevelOperations, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -42,15 +45,19 @@ object ResolveMergeIntoSchemaEvolution extends Rule[LogicalPlan] { if (changes.isEmpty) { m } else { - m transformUpWithNewOutput { - case r @ DataSourceV2Relation(_: SupportsRowLevelOperations, _, _, _, _, _) => + val finalAttrMapping = ArrayBuffer.empty[(Attribute, Attribute)] + val newTarget = m.targetTable.transform { + case r: DataSourceV2Relation => val referencedSourceSchema = MergeIntoTable.sourceSchemaForSchemaEvolution(m) val newTarget = performSchemaEvolution(r, referencedSourceSchema, changes) val oldTargetOutput = m.targetTable.output val newTargetOutput = newTarget.output val attributeMapping = oldTargetOutput.zip(newTargetOutput) - newTarget -> attributeMapping + finalAttrMapping ++= attributeMapping + newTarget } + val res = m.copy(targetTable = newTarget) + res.rewriteAttrs(AttributeMap(finalAttrMapping.toSeq)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 3f9e8da21d287..72274ee9bf174 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -916,19 +916,29 @@ case class MergeIntoTable( false } else { val actions = matchedActions ++ notMatchedActions - val assignments = actions.collect { - case a: UpdateAction => a.assignments - case a: InsertAction => a.assignments - }.flatten - val sourcePaths = DataTypeUtils.extractAllFieldPaths(sourceTable.schema) - assignments.forall { assignment => - assignment.resolved || - (assignment.value.resolved && sourcePaths.exists { - path => MergeIntoTable.isEqual(assignment, path) - }) + val hasStarActions = actions.exists { + case _: UpdateStarAction => true + case _: InsertStarAction => true + case _ => false + } + if (hasStarActions) { + // need to resolve star actions first + false + } else { + val assignments = actions.collect { + case a: UpdateAction => a.assignments + case a: InsertAction => a.assignments + }.flatten + val sourcePaths = DataTypeUtils.extractAllFieldPaths(sourceTable.schema) + assignments.forall { assignment => + assignment.resolved || + (assignment.value.resolved && sourcePaths.exists { + path => MergeIntoTable.isEqual(assignment, path) + }) } } } + } private lazy val sourceSchemaForEvolution: StructType = MergeIntoTable.sourceSchemaForSchemaEvolution(this) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index b7a8ff374b841..680fa63e09299 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -21,13 +21,14 @@ import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, In, Not} import org.apache.spark.sql.catalyst.optimizer.BuildLeft -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, ColumnDefaultValue, InMemoryTable, TableInfo} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, ColumnDefaultValue, Identifier, InMemoryTable, TableInfo} import org.apache.spark.sql.connector.expressions.{GeneralScalarExpression, LiteralValue} import org.apache.spark.sql.connector.write.MergeSummary import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.MergeRowsExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, CartesianProductExec} +import org.apache.spark.sql.functions.{array, col, lit, map, struct, substring} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, BooleanType, IntegerType, LongType, MapType, StringType, StructField, StructType} @@ -4411,7 +4412,8 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } - test("Merge schema evolution should evolve referencing new column assigned to something else") { + test("Merge schema evolution should not evolve when referencing new column" + + "assigned to something else") { Seq(true, false).foreach { withSchemaEvolution => withTempView("source") { createAndInitTable("pk INT NOT NULL, salary INT, dep STRING", @@ -5233,6 +5235,999 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"DROP TABLE IF EXISTS $tableNameAsString") } + test("merge with schema evolution using dataframe API: add new column and set all") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql(s"CREATE TABLE $tableNameAsString (pk INT NOT NULL, salary INT, dep STRING)") + + val targetData = Seq( + Row(1, 100, "hr"), + Row(2, 200, "software") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("salary", IntegerType), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("salary", IntegerType), + Column.create("dep", StringType), + Column.create("new_col", IntegerType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + sql(s"INSERT INTO $sourceTable VALUES (1, 101, 'support', 1)," + + s"(3, 301, 'support', 3), (4, 401, 'finance', 4)") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, + $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 101, "support", 1), + Row(2, 200, "software", null), + Row(3, 301, "support", 3), + Row(4, 401, "finance", 4))) + } else { + mergeBuilder.merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 101, "support"), + Row(2, 200, "software"), + Row(3, 301, "support"), + Row(4, 401, "finance"))) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution new column with set explicit column using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql(s"CREATE TABLE $tableNameAsString (pk INT NOT NULL, salary INT, dep STRING)") + + val targetData = Seq( + Row(1, 100, "hr"), + Row(2, 200, "software"), + Row(3, 300, "hr"), + Row(4, 400, "marketing"), + Row(5, 500, "executive") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("salary", IntegerType), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("salary", IntegerType), + Column.create("dep", StringType), + Column.create("active", BooleanType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + sql(s"INSERT INTO $sourceTable VALUES (4, 150, 'dummy', true)," + + s"(5, 250, 'dummy', true), (6, 350, 'dummy', false)") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .update(Map("dep" -> lit("software"), "active" -> col("source_table.active"))) + .whenNotMatched() + .insert(Map("pk" -> col("source_table.pk"), "salary" -> lit(0), + "dep" -> col("source_table.dep"), "active" -> col("source_table.active"))) + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, 100, "hr", null), + Row(2, 200, "software", null), + Row(3, 300, "hr", null), + Row(4, 400, "software", true), + Row(5, 500, "software", true), + Row(6, 0, "dummy", false))) + } else { + val e = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(e.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(e.getMessage.contains("A column, variable, or function parameter with name " + + "`active` cannot be resolved")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution add column with nested struct and set explicit columns " + + "using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING)""".stripMargin) + + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + val data = Seq( + Row(1, Row(10, Row(Array(3, 4), Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Array(4, 5), Map("e" -> "f"), true)), "engineering") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .update(Map( + "s.c1" -> lit(-1), + "s.c2.m" -> map(lit("k"), lit("v")), + "s.c2.a" -> array(lit(-1)), + "s.c2.c3" -> col("source_table.s.c2.c3"))) + .whenNotMatched() + .insert(Map( + "pk" -> col("source_table.pk"), + "s" -> struct( + col("source_table.s.c1").as("c1"), + struct( + col("source_table.s.c2.a").as("a"), + map(lit("g"), lit("h")).as("m"), + lit(true).as("c3") + ).as("c2") + ), + "dep" -> col("source_table.dep"))) + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq(Row(1, Row(-1, Row(Seq(-1), Map("k" -> "v"), false)), "hr"), + Row(2, Row(20, Row(Seq(4, 5), Map("g" -> "h"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == "FIELD_NOT_FOUND") + assert(exception.getMessage.contains("No such struct field `c3` in `a`, `m`. ")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution add column with nested struct and set all columns " + + "using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING)""".stripMargin) + + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + val data = Seq( + Row(1, Row(10, Row(Array(3, 4), Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Array(4, 5), Map("e" -> "f"), true)), "engineering") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq(Row(1, Row(10, Row(Seq(3, 4), Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Seq(4, 5), Map("e" -> "f"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution replace column with nested struct and " + + "set explicit columns using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING)""".stripMargin) + + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // removed column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + val data = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .update(Map( + "s.c1" -> lit(-1), + "s.c2.m" -> map(lit("k"), lit("v")), + "s.c2.a" -> array(lit(-1)), + "s.c2.c3" -> col("source_table.s.c2.c3"))) + .whenNotMatched() + .insert(Map( + "pk" -> col("source_table.pk"), + "s" -> struct( + col("source_table.s.c1").as("c1"), + struct( + array(lit(-2)).as("a"), + map(lit("g"), lit("h")).as("m"), + lit(true).as("c3") + ).as("c2") + ), + "dep" -> col("source_table.dep"))) + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq(Row(1, Row(-1, Row(Seq(-1), Map("k" -> "v"), false)), "hr"), + Row(2, Row(20, Row(Seq(-2), Map("g" -> "h"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == "FIELD_NOT_FOUND") + assert(exception.getMessage.contains("No such struct field `c3` in `a`, `m`. ")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution replace column with nested struct and set all columns " + + "using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + + val tableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) + .coalesce(1).writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // missing column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + val sourceData = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(Seq(1, 2), Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution replace column with nested struct and " + + "update top level struct using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, m: MAP>>, + |dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + + val tableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", ArrayType(IntegerType)), + StructField("m", MapType(StringType, StringType)) + ))) + ))), + StructField("dep", StringType) + )) + val targetData = Seq( + Row(1, Row(2, Row(Array(1, 2), Map("a" -> "b"))), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), tableSchema) + .coalesce(1).writeTo(tableNameAsString).append() + + // Create source table + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + // missing column 'a' + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) // new column + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + val sourceData = Seq( + Row(1, Row(10, Row(Map("c" -> "d"), false)), "sales"), + Row(2, Row(20, Row(Map("e" -> "f"), true)), "engineering") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("m", MapType(StringType, StringType)), + StructField("c3", BooleanType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(sourceData), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .update(Map("s" -> col("source_table.s"))) + .whenNotMatched() + .insertAll() + + if (withSchemaEvolution) { + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "hr"), + Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write extra fields `c3` to the struct `s`.`c2`")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge schema evolution should not evolve referencing new column " + + "via transform using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + sql(s"CREATE TABLE $tableNameAsString (pk INT NOT NULL, salary INT, dep STRING)") + + val targetData = Seq( + Row(1, 100, "hr"), + Row(2, 200, "software") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("salary", IntegerType), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("salary", IntegerType), + Column.create("dep", StringType), + Column.create("extra", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + sql(s"INSERT INTO $sourceTable VALUES (2, 150, 'dummy', 'blah')," + + s"(3, 250, 'dummy', 'blah')") + + val e = intercept[org.apache.spark.sql.AnalysisException] { + val builder = spark.table(sourceTable) + .mergeInto(tableNameAsString, + $"source_table.pk" === col(tableNameAsString + ".pk")) + + val builderWithEvolution = if (withSchemaEvolution) { + builder.withSchemaEvolution() + } else { + builder + } + + builderWithEvolution + .whenMatched() + .update(Map("extra" -> substring(col("source_table.extra"), 1, 2))) + .merge() + } + assert(e.errorClass.get == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(e.getMessage.contains( + "A column, variable, or function parameter with name " + + "`extra` cannot be resolved")) + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + + test("merge into with source missing fields in top-level struct using dataframe API") { + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + // Target table has struct with 3 fields at top level + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT, + |dep STRING)""".stripMargin) + + val targetData = Seq( + Row(0, Row(1, "a", true), "sales") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType), + StructField("c3", BooleanType) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + // Create source table with struct having only 2 fields (c1, c2) - missing c3 + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType)))), // missing c3 field + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + val data = Seq( + Row(1, Row(10, "b"), "hr"), + Row(2, Row(20, "c"), "engineering") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType)))), + StructField("dep", StringType))) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .merge() + + // Missing field c3 should be filled with NULL + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a", true), "sales"), + Row(1, Row(10, "b", null), "hr"), + Row(2, Row(20, "c", null), "engineering"))) + + sql(s"DROP TABLE $tableNameAsString") + } + } + + test("merge with null struct with missing nested field using dataframe API") { + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + // Target table has nested struct with fields c1 and c2 + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT>, + |dep STRING)""".stripMargin) + + val targetData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + // Create source table with missing nested field 'b' + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + // Source table has null for the nested struct + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, + $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + + if (coerceNestedTypes) { + mergeBuilder.merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } else { + // Without coercion, the merge should fail due to missing field + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot find data for the output column `s`.`c2`.`b`.")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + } + + test("merge null struct with schema evolution - " + + "source with missing and extra nested fields using dataframe API") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + val sourceTable = "cat.ns1.source_table" + withTable(sourceTable) { + // Target table has nested struct with fields c1 and c2 + sql( + s"""CREATE TABLE $tableNameAsString ( + |pk INT NOT NULL, + |s STRUCT>, + |dep STRING)""".stripMargin) + + val targetData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + val targetSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetSchema) + .writeTo(tableNameAsString).append() + + // Create source table with missing field 'b' and extra field 'c' in nested struct + val sourceIdent = Identifier.of(Array("ns1"), "source_table") + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + // missing field 'b' + StructField("c", StringType) // extra field 'c' + ))) + ))), + Column.create("dep", StringType)) + val tableInfo = new TableInfo.Builder() + .withColumns(columns) + .withProperties(extraTableProps) + .build() + catalog.createTable(sourceIdent, tableInfo) + + // Source data has null for the nested struct + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("c", StringType) + ))) + ))), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source_temp") + + sql(s"INSERT INTO $sourceTable SELECT * FROM source_temp") + + val mergeBuilder = spark.table(sourceTable) + .mergeInto(tableNameAsString, $"source_table.pk" === col(tableNameAsString + ".pk")) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + + if (coerceNestedTypes) { + if (withSchemaEvolution) { + // extra nested field is added + mergeBuilder.withSchemaEvolution().merge() + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x", null)), "sales"), + Row(1, null, "engineering"), + Row(2, null, "finance"))) + } else { + // extra nested field is not added + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot write extra fields `c` to the struct `s`.`c2`")) + } + } else { + // Without source struct coercion, the merge should fail + val exception = intercept[org.apache.spark.sql.AnalysisException] { + mergeBuilder.merge() + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains( + "Cannot write incompatible data for the table ``: " + + "Cannot find data for the output column `s`.`c2`.`b`.")) + } + + sql(s"DROP TABLE $tableNameAsString") + } + } + } + } + } + test("Merge schema evolution should error on non-existent column in UPDATE and INSERT") { withTable(tableNameAsString) { withTempView("source") { From bc427f724c4513bd883ef4665a4f10b52ebabe44 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Tue, 25 Nov 2025 22:19:38 -0800 Subject: [PATCH 225/400] [SPARK-54517][PYTHON][TESTS] Added utility decorators for Spark Connect parity tests ### What changes were proposed in this pull request? This PR aims to add utility decorators in Spark Connect cross version tests. - `skip_if_server_version_is` - `skip_if_server_version_is_greater_than_or_equal_to` ### Why are the changes needed? This is a forward-port from #53222. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53223 from ueshin/issues/SPARK-54517/decorators. Authored-by: Takuya Ueshin Signed-off-by: Dongjoon Hyun (cherry picked from commit d9dcb5c4dbb5548c773d147fcb6e7bd54a33ef07) Signed-off-by: Dongjoon Hyun --- python/pyspark/testing/connectutils.py | 30 +++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index 8539e16f03fc1..f53ac77b24b68 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -21,11 +21,10 @@ import unittest import uuid import contextlib +from typing import Callable, Optional from pyspark import Row, SparkConf -from pyspark.util import is_remote_only -from pyspark.testing.utils import PySparkErrorTestUtils -from pyspark import Row, SparkConf +from pyspark.loose_version import LooseVersion from pyspark.util import is_remote_only from pyspark.testing.utils import ( have_pandas, @@ -306,3 +305,28 @@ def _both_conf(): yield return _both_conf() + + +def skip_if_server_version_is( + cond: Callable[[LooseVersion], bool], reason: Optional[str] = None +) -> Callable[[...], ...]: + def decorator(f: Callable) -> Callable: + @functools.wraps(f) + def wrapper(self, *args, **kwargs): + version = self.spark.version + if cond(LooseVersion(version)): + raise unittest.SkipTest( + f"Skipping test {f.__name__} because server version is {version}" + + (f" ({reason})" if reason else "") + ) + return f(self, *args, **kwargs) + + return wrapper + + return decorator + + +def skip_if_server_version_is_greater_than_or_equal_to( + version: str, reason: Optional[str] = None +) -> Callable[[...], ...]: + return skip_if_server_version_is(lambda v: v >= LooseVersion(version), reason) From 18fdd026bd2e201e43297845a6c28968b51429b8 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Wed, 26 Nov 2025 14:51:21 +0800 Subject: [PATCH 226/400] [SPARK-54523][SQL] Fix default resolution during variant pushdown ### What changes were proposed in this pull request? [This PR](https://github.com/apache/spark/pull/53164) enables shredding and variant logical type annotation configs by default. However, some test suites assume the old behavior. This PR fixes those tests to also work with the new default configs. This PR also fixes a bug we discovered in the previous PR where variant default resolution would fail when pushVariantIntoScan was enabled. ### Why are the changes needed? To fix the bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53224 from harshmotw-db/harshmotw-db/shredding_fixes. Lead-authored-by: Harsh Motwani Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit d36bd625332cacbd33102de4ab2d0c9574d6de12) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Cast.scala | 25 ++++++++ .../util/ResolveDefaultColumnsUtil.scala | 2 +- .../org/apache/spark/sql/VariantSuite.scala | 64 +++++++++++-------- .../ParquetVariantShreddingSuite.scala | 12 ++-- .../parquet/VariantInferShreddingSuite.scala | 3 + 5 files changed, 73 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1f2805ec27899..1162a5394221c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -360,6 +360,31 @@ object Cast extends QueryErrorsBase { */ def canUpCast(from: DataType, to: DataType): Boolean = UpCastRule.canUpCast(from, to) + /** + * Returns true iff it is safe to provide a default value of `from` type typically defined in the + * data source metadata to the `to` type typically in the read schema of a query. + */ + def canAssignDefaultValue(from: DataType, to: DataType): Boolean = { + def isVariantStruct(st: StructType): Boolean = { + st.fields.length > 0 && st.fields.forall(_.metadata.contains("__VARIANT_METADATA_KEY")) + } + (from, to) match { + case (s1: StructType, s2: StructType) => + s1.length == s2.length && s1.fields.zip(s2.fields).forall { + case (f1, f2) => resolvableNullability(f1.nullable, f2.nullable) && + canAssignDefaultValue(f1.dataType, f2.dataType) + } + case (ArrayType(fromType, fn), ArrayType(toType, tn)) => + resolvableNullability(fn, tn) && canAssignDefaultValue(fromType, toType) + case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => + resolvableNullability(fn, tn) && canAssignDefaultValue(fromKey, toKey) && + canAssignDefaultValue(fromValue, toValue) + // A VARIANT field can be read as StructType due to shredding. + case (VariantType, s: StructType) => isVariantStruct(s) + case _ => canUpCast(from, to) + } + } + /** * Returns true iff we can cast the `from` type to `to` type as per the ANSI SQL. * In practice, the behavior is mostly the same as PostgreSQL. It disallows certain unreasonable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 4bef21d0a091e..488d1acf43ac4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -480,7 +480,7 @@ object ResolveDefaultColumns extends QueryErrorsBase val ret = analyzed match { case equivalent if equivalent.dataType == supplanted => equivalent - case canUpCast if Cast.canUpCast(canUpCast.dataType, supplanted) => + case _ if Cast.canAssignDefaultValue(analyzed.dataType, supplanted) => Cast(analyzed, supplanted, Some(conf.sessionLocalTimeZone)) case other => defaultValueFromWiderTypeLiteral(other, supplanted, colName).getOrElse( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala index ac6a4e435709f..16be9558409c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala @@ -197,36 +197,39 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval } test("round trip tests") { - val rand = new Random(42) - val input = Seq.fill(50) { - if (rand.nextInt(10) == 0) { - null - } else { - val value = new Array[Byte](rand.nextInt(50)) - rand.nextBytes(value) - val metadata = new Array[Byte](rand.nextInt(50)) - rand.nextBytes(metadata) - // Generate a valid metadata, otherwise the shredded reader will fail. - new VariantVal(value, Array[Byte](VERSION, 0, 0) ++ metadata) + withSQLConf(SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key -> "false") { + val rand = new Random(42) + val input = Seq.fill(50) { + if (rand.nextInt(10) == 0) { + null + } else { + val value = new Array[Byte](rand.nextInt(50)) + rand.nextBytes(value) + val metadata = new Array[Byte](rand.nextInt(50)) + rand.nextBytes(metadata) + // Generate a valid metadata, otherwise the shredded reader will fail. + new VariantVal(value, Array[Byte](VERSION, 0, 0) ++ metadata) + } } - } - val df = spark.createDataFrame( - spark.sparkContext.parallelize(input.map(Row(_))), - StructType.fromDDL("v variant") - ) - val result = df.collect().map(_.get(0).asInstanceOf[VariantVal]) + val df = spark.createDataFrame( + spark.sparkContext.parallelize(input.map(Row(_))), + StructType.fromDDL("v variant") + ) + val result = df.collect().map(_.get(0).asInstanceOf[VariantVal]) - def prepareAnswer(values: Seq[VariantVal]): Seq[String] = { - values.map(v => if (v == null) "null" else v.debugString()).sorted - } - assert(prepareAnswer(input) == prepareAnswer(result.toImmutableArraySeq)) + def prepareAnswer(values: Seq[VariantVal]): Seq[String] = { + values.map(v => if (v == null) "null" else v.debugString()).sorted + } + assert(prepareAnswer(input) == prepareAnswer(result.toImmutableArraySeq)) - withTempDir { dir => - val tempDir = new File(dir, "files").getCanonicalPath - df.write.parquet(tempDir) - val readResult = spark.read.parquet(tempDir).collect().map(_.get(0).asInstanceOf[VariantVal]) - assert(prepareAnswer(input) == prepareAnswer(readResult.toImmutableArraySeq)) + withTempDir { dir => + val tempDir = new File(dir, "files").getCanonicalPath + df.write.parquet(tempDir) + val readResult = spark.read.parquet(tempDir).collect() + .map(_.get(0).asInstanceOf[VariantVal]) + assert(prepareAnswer(input) == prepareAnswer(readResult.toImmutableArraySeq)) + } } } @@ -383,14 +386,19 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval ) cases.foreach { case (structDef, condition, parameters) => Seq(false, true).foreach { vectorizedReader => - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorizedReader.toString) { + withSQLConf( + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorizedReader.toString, + // Invalid variant binary fails during shredding schema inference. + SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key -> "false" + ) { withTempDir { dir => val file = new File(dir, "dir").getCanonicalPath val df = spark.sql(s"select $structDef as v from range(10)") df.write.parquet(file) val schema = StructType(Seq(StructField("v", VariantType))) val result = spark.read.schema(schema).parquet(file).selectExpr("to_json(v)") - val e = withSQLConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> "false") { + val e = withSQLConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> "false", + SQLConf.PUSH_VARIANT_INTO_SCAN.key -> "false") { intercept[org.apache.spark.SparkException](result.collect()) } checkError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala index 77140c1a91ee0..1f06ddb29bd47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala @@ -48,7 +48,8 @@ class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with Share test("timestamp physical type") { ParquetOutputTimestampType.values.foreach { timestampParquetType => - withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> timestampParquetType.toString) { + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> timestampParquetType.toString, + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> "true") { withTempDir { dir => val schema = "t timestamp, st struct, at array" val fullSchema = "v struct withSQLConf(SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key -> "true", - SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> ignoreVariantAnnotation.toString + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> ignoreVariantAnnotation.toString, + SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key -> "false" ) { withTempDir { dir => // write parquet file @@ -302,7 +304,8 @@ class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with Share "c struct>>" withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString, SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> true.toString, - SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema) { + SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema, + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> true.toString) { df.write.mode("overwrite").parquet(dir.getAbsolutePath) @@ -441,7 +444,8 @@ class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with Share "m map>" withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString, SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> true.toString, - SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema) { + SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema, + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> true.toString) { df.write.mode("overwrite").parquet(dir.getAbsolutePath) // Verify that we can read the full variant. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/VariantInferShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/VariantInferShreddingSuite.scala index cdaf6c488dc2a..49a43fffafb34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/VariantInferShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/VariantInferShreddingSuite.scala @@ -41,6 +41,9 @@ class VariantInferShreddingSuite extends QueryTest with SharedSparkSession with super.sparkConf.set(SQLConf.PUSH_VARIANT_INTO_SCAN.key, "true") .set(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key, "true") .set(SQLConf.VARIANT_INFER_SHREDDING_SCHEMA.key, "true") + // We cannot check the physical shredding schemas if the variant logical type annotation is + // used + .set(SQLConf.PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE.key, "false") } private def withTempTable(tableNames: String*)(f: => Unit): Unit = { From 663cb80dc5e4b96e5ee0fc806f53c90e4c0752f6 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 26 Nov 2025 19:48:29 +0800 Subject: [PATCH 227/400] [SPARK-54524][BUILD] Fix Connect JDBC driver dependencies ### What changes were proposed in this pull request? The Connect JDBC driver is built on top of the Connect JVM client, so it should declare the Connect JVM client as the dependency, and Maven will resolve transitive deps correctly. Note, many RDBMS products provide a single no-deps JDBC driver jar to simplify downstream projects' integration, while this is not achieved at the current stage, and may be considered in the future. ### Why are the changes needed? Fix Connect JDBC driver dependency management. For example, it should not pull protobuf libs which are already shaded in connect-jdbc-client. https://mvnrepository.com/artifact/org.apache.spark/spark-connect-client-jdbc_2.13/4.1.0-preview4 ### Does this PR introduce _any_ user-facing change? No, the Connect JDBC driver is not released yet. ### How was this patch tested? Pass GHA, plus manual verifications. - Checked the effective `pom.xml` which is located at `sql/connect/client/jdbc/dependency-reduced-pom.xml` after compiling, only one compile scope dep. ```xml org.apache.spark spark-connect-client-jvm_2.13 4.2.0-SNAPSHOT compile ``` - Verified locally by running tests using Maven/SBT Maven ``` build/mvn -Phive,hive-thriftserver clean install -DskipTests build/mvn -Phive,hive-thriftserver -pl sql/connect/client/jdbc test ``` SBT ``` build/sbt -Phive,hive-thriftserver "connect-client-jdbc/test" ``` - Verified CLI cases with BeeLine by creating a binary release artifact using Maven/SBT Launch a Connect Server first, then ``` dev/make-distribution.sh [--sbt-enabled] -Phive,hive-thriftserver cd dist SPARK_CONNECT_BEELINE=1 bin/beeline -u jdbc:sc://localhost:15002 -e "select 'Hello, Spark Connect!', version() as server_version;" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53230 from pan3793/SPARK-53484. Authored-by: Cheng Pan Signed-off-by: yangjie01 (cherry picked from commit b9dd02baed3e69e983c05d63ffbcab515e3d1c71) Signed-off-by: yangjie01 --- sql/connect/client/jdbc/pom.xml | 82 ++------------------------------- 1 file changed, 3 insertions(+), 79 deletions(-) diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index b2c5d4fa6fcb6..6dd373071f6c2 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -37,77 +37,8 @@ org.apache.spark - spark-connect-common_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sql-api_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-connect-shims_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sketch_${scala.binary.version} - ${project.version} - - - org.scala-lang - scala-compiler - compile - - - org.scala-lang.modules - scala-xml_${scala.binary.version} - compile - - - - com.google.protobuf - protobuf-java - compile - - - com.google.guava - guava - compile - - - com.google.guava - failureaccess - compile - - - org.apache.spark - spark-tags_${scala.binary.version} - test-jar - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-sql-api_${scala.binary.version} - ${project.version} - tests - test - - - org.apache.spark - spark-common-utils_${scala.binary.version} + spark-connect-client-jvm_${scala.binary.version} ${project.version} - tests - test org.apache.spark @@ -116,13 +47,6 @@ tests test - - - com.typesafe - mima-core_${scala.binary.version} - ${mima.version} - test - target/scala-${scala.binary.version}/classes @@ -142,10 +66,10 @@ maven-shade-plugin false - true + false - org.apache.spark:spark-connect-client-jdbc_${scala.binary.version} + org.spark-project.spark:unused From d2f35f5ddc2fd01a99425ef279aec8f49800406d Mon Sep 17 00:00:00 2001 From: Yicong-Huang <17627829+Yicong-Huang@users.noreply.github.com> Date: Wed, 26 Nov 2025 11:14:29 +0900 Subject: [PATCH 228/400] [SPARK-54520][INFRA] Ignore Missing Packages in CI free_disk_space Step ### What changes were proposed in this pull request? Allow the CI `free_disk_space` step to ignore missing packages during `apt-get remove`. ### Why are the changes needed? Intermittent network or upstream issues can cause apt-get to fail when certain packages cannot be located. This failure blocks the `apt-get remove` operations in `free_disk_space`. Since these packages are optional for cleanup and their absence is harmless, we can safely ignore missing-package errors and let the step continue. Example CI failure: [https://github.com/Yicong-Huang/spark/actions/runs/19686762867/job/56398651193](https://github.com/Yicong-Huang/spark/actions/runs/19686762867/job/56398651193) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #53226 from Yicong-Huang/fix/ignore-unfound-packages. Authored-by: Yicong-Huang <17627829+Yicong-Huang@users.noreply.github.com> Signed-off-by: Hyukjin Kwon (cherry picked from commit 78e911826d648b491732e032576d13958efa1fc3) Signed-off-by: Dongjoon Hyun --- dev/free_disk_space | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/free_disk_space b/dev/free_disk_space index 2e6621045901b..d0916a32f301a 100755 --- a/dev/free_disk_space +++ b/dev/free_disk_space @@ -44,6 +44,7 @@ sudo rm -rf /opt/hostedtoolcache/go sudo rm -rf /opt/hostedtoolcache/node du -sh /opt/* +sudo apt-get update --fix-missing sudo apt-get remove --purge -y '^aspnet.*' sudo apt-get remove --purge -y '^dotnet-.*' sudo apt-get remove --purge -y '^llvm-.*' From b3418d60f1d129f41837df68711cb8991a5cf1b9 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Wed, 26 Nov 2025 11:53:08 -0800 Subject: [PATCH 229/400] [SPARK-54517][PYTHON][TESTS][FOLLOWUP] Fix typehints using ellipsis ### What changes were proposed in this pull request? This is a follow-up of #53223. Fixes a typehint using ellipsis that are not available as type arguments in Python 3.10. ### Why are the changes needed? #53223 caused a test failure in Python 3.10 tests: ``` File "/__w/spark/spark/python/pyspark/testing/connectutils.py", line 312, in ) -> Callable[[...], ...]: ^^^^^^^^^^^^^^^^^^^^ File "/usr/local/pypy/pypy3.10/lib/pypy3.10/typing.py", line 1206, in __getitem__ return self.__getitem_inner__(params) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/usr/local/pypy/pypy3.10/lib/pypy3.10/typing.py", line 312, in inner return func(*args, **kwds) ^^^^^^^^^^^^^^^^^^^ File "/usr/local/pypy/pypy3.10/lib/pypy3.10/typing.py", line 1212, in __getitem_inner__ result = _type_check(result, msg) ^^^^^^^^^^^^^^^^^^^^^^^^ File "/usr/local/pypy/pypy3.10/lib/pypy3.10/typing.py", line 176, in _type_check raise TypeError(f"{msg} Got {arg!r:.100}.") TypeError: Callable[args, result]: result must be a type. Got Ellipsis. ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53238 from ueshin/issues/SPARK-54517/typehints. Authored-by: Takuya Ueshin Signed-off-by: Dongjoon Hyun (cherry picked from commit c11edf9e8d21417c4a44bcd1a35b1792f556af19) Signed-off-by: Dongjoon Hyun --- python/pyspark/testing/connectutils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index f53ac77b24b68..ee86f5c039744 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -309,7 +309,7 @@ def _both_conf(): def skip_if_server_version_is( cond: Callable[[LooseVersion], bool], reason: Optional[str] = None -) -> Callable[[...], ...]: +) -> Callable: def decorator(f: Callable) -> Callable: @functools.wraps(f) def wrapper(self, *args, **kwargs): @@ -328,5 +328,5 @@ def wrapper(self, *args, **kwargs): def skip_if_server_version_is_greater_than_or_equal_to( version: str, reason: Optional[str] = None -) -> Callable[[...], ...]: +) -> Callable: return skip_if_server_version_is(lambda v: v >= LooseVersion(version), reason) From 2128df84872367d4c4816e3d93243bd194b206b6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 26 Nov 2025 11:59:13 -0800 Subject: [PATCH 230/400] [SPARK-54528][CONNECT] Close URLClassLoader eagerly to avoid OOM ### What changes were proposed in this pull request? In Spark Connect, every client session has its own class loader in the server side (both driver and executors), for isolation purpose. However, when closing the session, we don't close the class loader. We rely on GC to close the class loader and release the resource, but it's unpredicatable and not efficient. When sessions are being created frequently, OOM may happen. This PR fixes the issue by closing URLClassLoader eagerly when closing a session. It also adds a config so that we can tune the session cache size in the executor side to save memory. With this patch, OOM is gone. ### Why are the changes needed? avoid OOM ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Our internal CI service has limited memory and `SparkConnectJdbcDataTypeSuite` keeps failing due to OOM. This test suite creates and closes sessions back to back frequently, and GC can't catch up to release the class loaders. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Cursor 2.1.36 Closes #53233 from cloud-fan/mem. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit e2649651d78d7d5817ceca729b136d5bf9a00ff9) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/executor/Executor.scala | 18 ++++++++++++++++-- .../apache/spark/internal/config/package.scala | 9 +++++++++ .../sql/connect/test/RemoteSparkSession.scala | 2 ++ .../spark/sql/artifact/ArtifactManager.scala | 15 ++++++++++++++- 4 files changed, 41 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index fc22107e008be..04e9662943361 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -20,7 +20,7 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} import java.lang.Thread.UncaughtExceptionHandler import java.lang.management.ManagementFactory -import java.net.{URI, URL} +import java.net.{URI, URL, URLClassLoader} import java.nio.ByteBuffer import java.util.{Locale, Properties} import java.util.concurrent._ @@ -212,7 +212,7 @@ private[spark] class Executor( val defaultSessionState: IsolatedSessionState = newSessionState(JobArtifactState("default", None)) val isolatedSessionCache: Cache[String, IsolatedSessionState] = CacheBuilder.newBuilder() - .maximumSize(100) + .maximumSize(conf.get(EXECUTOR_ISOLATED_SESSION_CACHE_SIZE)) .expireAfterAccess(30, TimeUnit.MINUTES) .removalListener(new RemovalListener[String, IsolatedSessionState]() { override def onRemoval( @@ -220,6 +220,20 @@ private[spark] class Executor( val state = notification.getValue // Cache is always used for isolated sessions. assert(!isDefaultState(state.sessionUUID)) + // Close the urlClassLoader to release resources. + try { + state.urlClassLoader match { + case urlClassLoader: URLClassLoader => + urlClassLoader.close() + logInfo(log"Closed urlClassLoader (URLClassLoader) for evicted session " + + log"${MDC(SESSION_ID, state.sessionUUID)}") + case _ => + } + } catch { + case NonFatal(e) => + logWarning(log"Failed to close urlClassLoader for session " + + log"${MDC(SESSION_ID, state.sessionUUID)}", e) + } val sessionBasedRoot = new File(SparkFiles.getRootDirectory(), state.sessionUUID) if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) { Utils.deleteRecursively(sessionBasedRoot) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 9876848f654a4..8b32c18aa3b61 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -358,6 +358,15 @@ package object config { .intConf .createWithDefault(60) + private[spark] val EXECUTOR_ISOLATED_SESSION_CACHE_SIZE = + ConfigBuilder("spark.executor.isolatedSessionCache.size") + .doc("Maximum number of isolated sessions to cache in the executor. Each cached session " + + "maintains its own classloader for artifact isolation.") + .version("4.1.0") + .intConf + .checkValue(_ > 0, "The cache size must be positive.") + .createWithDefault(100) + private[spark] val EXECUTOR_PROCESS_TREE_METRICS_ENABLED = ConfigBuilder("spark.executor.processTreeMetrics.enabled") .doc("Whether to collect process tree metrics (from the /proc filesystem) when collecting " + diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala index 6d8d2edcf0821..efac3bc7561fd 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala @@ -135,6 +135,8 @@ object SparkConnectServerUtils { "spark.connect.execute.reattachable.senderMaxStreamSize=123", // Testing SPARK-49673, setting maxBatchSize to 10MiB s"spark.connect.grpc.arrow.maxBatchSize=${10 * 1024 * 1024}", + // Cache less sessions to save memory. + "spark.executor.isolatedSessionCache.size=5", // Disable UI "spark.ui.enabled=false").flatMap(v => "--conf" :: v :: Nil) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index 5889fe581d4e0..346cdb832c3ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -450,7 +450,20 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging pythonIncludeList.clear() sparkContextRelativePaths.clear() - // Removed cached classloader + // Close and remove cached classloader + cachedClassLoader.foreach { + case urlClassLoader: URLClassLoader => + try { + urlClassLoader.close() + logDebug(log"Closed URLClassLoader for session " + + log"${MDC(LogKeys.SESSION_ID, session.sessionUUID)}") + } catch { + case e: IOException => + logWarning(log"Failed to close URLClassLoader for session " + + log"${MDC(LogKeys.SESSION_ID, session.sessionUUID)}", e) + } + case _ => + } cachedClassLoader = None } From a30d93b646c27fc419ffaba6458aca44c72f741a Mon Sep 17 00:00:00 2001 From: Yicong-Huang <17627829+Yicong-Huang@users.noreply.github.com> Date: Wed, 26 Nov 2025 15:28:19 -0800 Subject: [PATCH 231/400] [SPARK-53597][PYTHON][SQL][DOCS] Add `asTable()` and `TableArg` examples ### Why are the changes needed? There are no examples showing how to use `DataFrame.asTable()` and `TableArg` features. These examples are essential for users to understand how to: - Convert DataFrames to table arguments for use in UDTFs - Control data partitioning and ordering when passing DataFrames as table arguments - Use the various methods available on `TableArg` instances ### Does this PR introduce _any_ user-facing change? Yes, this is a documentation-only change that adds examples to the public API documentation. No functional changes are introduced. ### How was this patch tested? The examples follow the same patterns used in existing test cases (`test_udtf.py`). The examples are written in doctest format and should be validated when running doctests. ### Was this patch authored or co-authored using generative AI tooling? Co-Generated-by: Cursor Closes #53240 from Yicong-Huang/SPARK-53597/docs/add-astable-examples. Authored-by: Yicong-Huang <17627829+Yicong-Huang@users.noreply.github.com> Signed-off-by: Dongjoon Hyun (cherry picked from commit a3e55e5dc8ddef5f78dc947b274b407bc804c208) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/dataframe.py | 58 +++++++++++ python/pyspark/sql/table_arg.py | 166 ++++++++++++++++++++++++++++++++ 2 files changed, 224 insertions(+) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ca33539df960b..502883cf59b1a 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6691,6 +6691,64 @@ def asTable(self) -> TableArg: ------- :class:`table_arg.TableArg` A `TableArg` object representing a table argument. + + Examples + -------- + >>> from pyspark.sql.functions import udtf + >>> + >>> # Create a simple UDTF that processes table data + >>> @udtf(returnType="id: int, doubled: int") + ... class DoubleUDTF: + ... def eval(self, row): + ... yield row["id"], row["id"] * 2 + ... + >>> # Create a DataFrame + >>> df = spark.createDataFrame([(1,), (2,), (3,)], ["id"]) + >>> + >>> # Use asTable() to pass the DataFrame as a table argument to the UDTF + >>> result = DoubleUDTF(df.asTable()) + >>> result.show() + +---+-------+ + | id|doubled| + +---+-------+ + | 1| 2| + | 2| 4| + | 3| 6| + +---+-------+ + >>> + >>> # Use partitionBy and orderBy to control data partitioning and ordering + >>> df2 = spark.createDataFrame( + ... [(1, "a"), (1, "b"), (2, "c"), (2, "d")], ["key", "value"] + ... ) + >>> + >>> @udtf(returnType="key: int, value: string") + ... class ProcessUDTF: + ... def eval(self, row): + ... yield row["key"], row["value"] + ... + >>> # Partition by 'key' and order by 'value' within each partition + >>> result2 = ProcessUDTF(df2.asTable().partitionBy("key").orderBy("value")) + >>> result2.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 1| b| + | 2| c| + | 2| d| + +---+-----+ + >>> + >>> # Use withSinglePartition to process all data in a single partition + >>> result3 = ProcessUDTF(df2.asTable().withSinglePartition().orderBy("value")) + >>> result3.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 1| b| + | 2| c| + | 2| d| + +---+-----+ """ ... diff --git a/python/pyspark/sql/table_arg.py b/python/pyspark/sql/table_arg.py index f96b40b2dee1f..483b26eb97abd 100644 --- a/python/pyspark/sql/table_arg.py +++ b/python/pyspark/sql/table_arg.py @@ -40,6 +40,59 @@ class TableArg(TableValuedFunctionArgument): def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": """ Partitions the data based on the specified columns. + + This method partitions the table argument data by the specified columns. + It must be called before `orderBy()` and cannot be called after + `withSinglePartition()` has been called. + + Parameters + ---------- + cols : str, :class:`Column`, or list + Column names or :class:`Column` objects to partition by. + + Returns + ------- + :class:`TableArg` + A new `TableArg` instance with partitioning applied. + + Examples + -------- + >>> from pyspark.sql.functions import udtf + >>> + >>> @udtf(returnType="key: int, value: string") + ... class ProcessUDTF: + ... def eval(self, row): + ... yield row["key"], row["value"] + ... + >>> df = spark.createDataFrame( + ... [(1, "a"), (1, "b"), (2, "c"), (2, "d")], ["key", "value"] + ... ) + >>> + >>> # Partition by a single column + >>> result = ProcessUDTF(df.asTable().partitionBy("key")) + >>> result.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 1| b| + | 2| c| + | 2| d| + +---+-----+ + >>> + >>> # Partition by multiple columns + >>> df2 = spark.createDataFrame( + ... [(1, "x", 10), (1, "x", 20), (2, "y", 30)], ["key", "category", "value"] + ... ) + >>> result2 = ProcessUDTF(df2.asTable().partitionBy("key", "category")) + >>> result2.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| x| + | 1| x| + | 2| y| + +---+-----+ """ ... @@ -47,6 +100,72 @@ def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": def orderBy(self, *cols: "ColumnOrName") -> "TableArg": """ Orders the data within each partition by the specified columns. + + This method orders the data within partitions. It must be called after + `partitionBy()` or `withSinglePartition()` has been called. + + Parameters + ---------- + cols : str, :class:`Column`, or list + Column names or :class:`Column` objects to order by. Columns can be + ordered in ascending or descending order using :meth:`Column.asc` or + :meth:`Column.desc`. + + Returns + ------- + :class:`TableArg` + A new `TableArg` instance with ordering applied. + + Examples + -------- + >>> from pyspark.sql.functions import udtf + >>> + >>> @udtf(returnType="key: int, value: string") + ... class ProcessUDTF: + ... def eval(self, row): + ... yield row["key"], row["value"] + ... + >>> df = spark.createDataFrame( + ... [(1, "b"), (1, "a"), (2, "d"), (2, "c")], ["key", "value"] + ... ) + >>> + >>> # Order by a single column within partitions + >>> result = ProcessUDTF(df.asTable().partitionBy("key").orderBy("value")) + >>> result.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 1| b| + | 2| c| + | 2| d| + +---+-----+ + >>> + >>> # Order by multiple columns + >>> df2 = spark.createDataFrame( + ... [(1, "a", 2), (1, "a", 1), (1, "b", 3)], ["key", "value", "num"] + ... ) + >>> result2 = ProcessUDTF(df2.asTable().partitionBy("key").orderBy("value", "num")) + >>> result2.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 1| a| + | 1| b| + +---+-----+ + >>> + >>> # Order by descending order + >>> result3 = ProcessUDTF(df.asTable().partitionBy("key").orderBy(df.value.desc())) + >>> result3.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| b| + | 1| a| + | 2| d| + | 2| c| + +---+-----+ """ ... @@ -54,5 +173,52 @@ def orderBy(self, *cols: "ColumnOrName") -> "TableArg": def withSinglePartition(self) -> "TableArg": """ Forces the data to be processed in a single partition. + + This method indicates that all data should be treated as a single partition. + It cannot be called after `partitionBy()` has been called. `orderBy()` can + be called after this method to order the data within the single partition. + + Returns + ------- + :class:`TableArg` + A new `TableArg` instance with single partition constraint applied. + + Examples + -------- + >>> from pyspark.sql.functions import udtf + >>> + >>> @udtf(returnType="key: int, value: string") + ... class ProcessUDTF: + ... def eval(self, row): + ... yield row["key"], row["value"] + ... + >>> df = spark.createDataFrame( + ... [(1, "a"), (2, "b"), (3, "c")], ["key", "value"] + ... ) + >>> + >>> # Process all data in a single partition + >>> result = ProcessUDTF(df.asTable().withSinglePartition()) + >>> result.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 2| b| + | 3| c| + +---+-----+ + >>> + >>> # Use withSinglePartition and orderBy together + >>> df2 = spark.createDataFrame( + ... [(3, "c"), (1, "a"), (2, "b")], ["key", "value"] + ... ) + >>> result2 = ProcessUDTF(df2.asTable().withSinglePartition().orderBy("key")) + >>> result2.show() + +---+-----+ + |key|value| + +---+-----+ + | 1| a| + | 2| b| + | 3| c| + +---+-----+ """ ... From db2d19fadb290fe214b8918c982cb203e3552bd2 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Thu, 27 Nov 2025 16:45:18 -0800 Subject: [PATCH 232/400] [SPARK-54522][SQL][TEST] Variant test suite fixes for shredding configs ### What changes were proposed in this pull request? This PR makes minor changes to test suites so that they pass with variant shredding-related configs enabled, without hurting the purpose of the tests. ### Why are the changes needed? These changes would help tests pass if shredding configs were enabled by default in the future. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53242 from harshmotw-db/harshmotw-db/suite_fixes. Authored-by: Harsh Motwani Signed-off-by: Dongjoon Hyun (cherry picked from commit 5b0a667d9a5cf1f7fe1107844807110dec5fc553) Signed-off-by: Dongjoon Hyun --- .../src/test/scala/org/apache/spark/sql/VariantSuite.scala | 4 +++- .../datasources/parquet/ParquetVariantShreddingSuite.scala | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala index 16be9558409c3..6510a165e2589 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala @@ -820,7 +820,9 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval // The initial size of the buffer backing a cached dataframe column is 128KB. // See `ColumnBuilder`. val numKeys = 128 * 1024 - val keyIterator = (0 until numKeys).iterator + // We start in long range because the shredded writer writes int64 by default which wouldn't + // match narrower binaries. + val keyIterator = (Int.MaxValue + 1L until Int.MaxValue + 1L + numKeys).iterator val entries = Array.fill(numKeys)(s"""\"${keyIterator.next()}\": \"test\"""") val jsonStr = s"{${entries.mkString(", ")}}" val query = s"""select named_struct( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala index 1f06ddb29bd47..1cc6d3afbee59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala @@ -379,7 +379,8 @@ class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with Share "struct>>" withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString, SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> true.toString, - SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema) { + SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema, + SQLConf.PARQUET_IGNORE_VARIANT_ANNOTATION.key -> true.toString) { df.write.mode("overwrite").parquet(dir.getAbsolutePath) // Verify that we can read the full variant. From 681542b5644ab53445dba3188d9f9c451aeb43ba Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Thu, 27 Nov 2025 16:47:33 -0800 Subject: [PATCH 233/400] [SPARK-54537][CONNECT] Fix SparkConnectDatabaseMetaData getSchemas/getTables on catalogs missing namespace capbalitity ### What changes were proposed in this pull request? For DSv2 catalog implementation that does not mix in the interface `SupportsNamespaces`, `SHOW SCHEMAS IN foo_catalog` fails with `MISSING_CATALOG_ABILITY.NAMESPACES`, we should catch this and return empty result instead of failing on `SparkConnectDatabaseMetaData#getSchemas|getTables` ### Why are the changes needed? Fix a bug in the Connect JDBC dirver. ### Does this PR introduce _any_ user-facing change? No, Connect JDBC dirver is an unreleased feature. ### How was this patch tested? UT is adjusted to cover the changes. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53246 from pan3793/SPARK-54537. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 0a76a2138286b9b54fbde2a71a30e7d387c823d2) Signed-off-by: Dongjoon Hyun --- .../jdbc/SparkConnectDatabaseMetaData.scala | 28 +++++++++++-------- .../SparkConnectDatabaseMetaDataSuite.scala | 15 ++++++++++ 2 files changed, 32 insertions(+), 11 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala index 490fb7b6472ea..7a37c272daf2b 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaData.scala @@ -326,23 +326,29 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas case Some(schemaPattern) => $"TABLE_SCHEM".like(schemaPattern) } + lazy val emptyDf = conn.spark.emptyDataFrame + .withColumn("TABLE_SCHEM", lit("")) + .withColumn("TABLE_CATALOG", lit("")) + def internalGetSchemas( catalogOpt: Option[String], schemaFilterExpr: Column): connect.DataFrame = { val catalog = catalogOpt.getOrElse(conn.getCatalog) - // Spark SQL supports LIKE clause in SHOW SCHEMAS command, but we can't use that - // because the LIKE pattern does not follow SQL standard. - conn.spark.sql(s"SHOW SCHEMAS IN ${quoteIdentifier(catalog)}") - .select($"namespace".as("TABLE_SCHEM")) - .filter(schemaFilterExpr) - .withColumn("TABLE_CATALOG", lit(catalog)) + try { + // Spark SQL supports LIKE clause in SHOW SCHEMAS command, but we can't use that + // because the LIKE pattern does not follow SQL standard. + conn.spark.sql(s"SHOW SCHEMAS IN ${quoteIdentifier(catalog)}") + .select($"namespace".as("TABLE_SCHEM")) + .filter(schemaFilterExpr) + .withColumn("TABLE_CATALOG", lit(catalog)) + } catch { + case st: SparkThrowable if st.getCondition == "MISSING_CATALOG_ABILITY.NAMESPACES" => + emptyDf + } } if (catalog == null) { // search in all catalogs - val emptyDf = conn.spark.emptyDataFrame - .withColumn("TABLE_SCHEM", lit("")) - .withColumn("TABLE_CATALOG", lit("")) conn.spark.catalog.listCatalogs().collect().map(_.name).map { c => internalGetSchemas(Some(c), schemaFilterExpr) }.fold(emptyDf) { (l, r) => l.unionAll(r) } @@ -402,7 +408,7 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas $"TABLE_NAME".like(tableNamePattern) } - val emptyDf = conn.spark.emptyDataFrame + lazy val emptyDf = conn.spark.emptyDataFrame .withColumn("TABLE_CAT", lit("")) .withColumn("TABLE_SCHEM", lit("")) .withColumn("TABLE_NAME", lit("")) @@ -490,7 +496,7 @@ class SparkConnectDatabaseMetaData(conn: SparkConnectConnection) extends Databas $"COLUMN_NAME".like(columnNamePattern) } - val emptyDf = conn.spark.emptyDataFrame + lazy val emptyDf = conn.spark.emptyDataFrame .withColumn("TABLE_CAT", lit("")) .withColumn("TABLE_SCHEM", lit("")) .withColumn("TABLE_NAME", lit("")) diff --git a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala index 255537af4bbb4..4d66392109e70 100644 --- a/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala +++ b/sql/connect/client/jdbc/src/test/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectDatabaseMetaDataSuite.scala @@ -34,6 +34,8 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark // catalyst test jar is inaccessible here, but presents at the testing connect server classpath private val TEST_IN_MEMORY_CATALOG = "org.apache.spark.sql.connector.catalog.InMemoryCatalog" + private val TEST_BASIC_IN_MEMORY_CATALOG = + "org.apache.spark.sql.connector.catalog.BasicInMemoryTableCatalog" private def registerCatalog( name: String, className: String)(implicit spark: SparkSession): Unit = { @@ -254,6 +256,12 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark withConnection { conn => implicit val spark: SparkSession = conn.asInstanceOf[SparkConnectConnection].spark + // this catalog does not support namespace + registerCatalog("test_noop", TEST_BASIC_IN_MEMORY_CATALOG) + // Spark loads catalog plugins lazily, we must initialize it first, + // otherwise it won't be listed by SHOW CATALOGS + conn.setCatalog("test_noop") + registerCatalog("test`cat", TEST_IN_MEMORY_CATALOG) spark.sql("CREATE DATABASE IF NOT EXISTS `test``cat`.t_db1") @@ -290,6 +298,7 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark } // list schemas in current catalog + conn.setCatalog("spark_catalog") assert(conn.getCatalog === "spark_catalog") val getSchemasInCurrentCatalog = List(null, "%").map { database => () => metadata.getSchemas("", database) } @@ -405,6 +414,12 @@ class SparkConnectDatabaseMetaDataSuite extends ConnectFunSuite with RemoteSpark withConnection { conn => implicit val spark: SparkSession = conn.asInstanceOf[SparkConnectConnection].spark + // this catalog does not support namespace + registerCatalog("test_noop", TEST_BASIC_IN_MEMORY_CATALOG) + // Spark loads catalog plugins lazily, we must initialize it first, + // otherwise it won't be listed by SHOW CATALOGS + conn.setCatalog("test_noop") + // this catalog does not support view registerCatalog("testcat", TEST_IN_MEMORY_CATALOG) From eefe0ca49474544f47abb08153f86a331b0f6810 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Thu, 27 Nov 2025 16:49:02 -0800 Subject: [PATCH 234/400] [SPARK-54540][CONNECT] Couples of minor fix for Connect JDBC driver ### What changes were proposed in this pull request? Couples of minor fix for Connect JDBC driver - fix typos - use import instead of the inline full package class reference - a missing call `checkOpen` - remove duplicated scaladocs generated by AI. ### Why are the changes needed? Minor fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53248 from pan3793/SPARK-54540. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 68653991a4ddc499e8d7fd265ecebee2aa4c77b4) Signed-off-by: Dongjoon Hyun --- .../client/jdbc/SparkConnectConnection.scala | 4 +- .../client/jdbc/SparkConnectResultSet.scala | 60 ++++++++++--------- .../client/jdbc/util/JdbcErrorUtils.scala | 4 +- 3 files changed, 36 insertions(+), 32 deletions(-) diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectConnection.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectConnection.scala index 95ec956771dbb..21b9471bb6069 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectConnection.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectConnection.scala @@ -185,7 +185,7 @@ class SparkConnectConnection(val url: String, val info: Properties) extends Conn if (level != Connection.TRANSACTION_NONE) { throw new SQLFeatureNotSupportedException( "Requested transaction isolation level " + - s"${stringfiyTransactionIsolationLevel(level)} is not supported") + s"${stringifyTransactionIsolationLevel(level)} is not supported") } } @@ -207,7 +207,7 @@ class SparkConnectConnection(val url: String, val info: Properties) extends Conn override def setHoldability(holdability: Int): Unit = { if (holdability != ResultSet.HOLD_CURSORS_OVER_COMMIT) { throw new SQLFeatureNotSupportedException( - s"Holdability ${stringfiyHoldability(holdability)} is not supported") + s"Holdability ${stringifyHoldability(holdability)} is not supported") } } diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala index 8b2d4578d0aa6..0070cbd93c3eb 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/SparkConnectResultSet.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.connect.client.jdbc import java.io.{InputStream, Reader} import java.net.URL import java.sql.{Array => JdbcArray, _} -import java.time.LocalTime +import java.time.{LocalDateTime, LocalTime} +import java.time.temporal.ChronoUnit import java.util import java.util.Calendar @@ -44,9 +45,15 @@ class SparkConnectResultSet( private var cursor: Int = 0 private var _wasNull: Boolean = false - override def wasNull: Boolean = _wasNull + + override def wasNull: Boolean = { + checkOpen() + _wasNull + } override def next(): Boolean = { + checkOpen() + val hasNext = iterator.hasNext if (hasNext) { currentRow = iterator.next() @@ -60,7 +67,7 @@ class SparkConnectResultSet( hasNext } - @volatile protected var closed: Boolean = false + @volatile private var closed: Boolean = false override def isClosed: Boolean = closed @@ -97,6 +104,8 @@ class SparkConnectResultSet( } override def findColumn(columnLabel: String): Int = { + checkOpen() + sparkResult.schema.getFieldIndex(columnLabel) match { case Some(i) => i + 1 case None => @@ -156,9 +165,8 @@ class SparkConnectResultSet( // Note: java.sql.Time can only store up to millisecond precision (3 digits). // For TIME types with higher precision (TIME(4-9)), microseconds/nanoseconds are truncated. // If user needs full precision, - // should use: getObject(columnIndex, classOf[java.time.LocalTime]) - val millisSinceMidnight = - java.time.temporal.ChronoUnit.MILLIS.between(LocalTime.MIDNIGHT, localTime) + // should use: getObject(columnIndex, classOf[LocalTime]) + val millisSinceMidnight = ChronoUnit.MILLIS.between(LocalTime.MIDNIGHT, localTime) new Time(millisSinceMidnight) } } @@ -172,9 +180,9 @@ class SparkConnectResultSet( sparkResult.schema.fields(idx).dataType match { case TimestampNTZType => // TIMESTAMP_NTZ is represented as LocalDateTime - Timestamp.valueOf(value.asInstanceOf[java.time.LocalDateTime]) + Timestamp.valueOf(value.asInstanceOf[LocalDateTime]) case TimestampType => - // TIMESTAMP is represented as java.sql.Timestamp + // TIMESTAMP is represented as Timestamp value.asInstanceOf[Timestamp] case other => throw new SQLException( @@ -533,52 +541,46 @@ class SparkConnectResultSet( throw new SQLFeatureNotSupportedException /** - * Gets the value of the designated column in the current row as a java.sql.Date object. - * The Calendar parameter is ignored for Date type since it is not timezone-aware. + * @inheritdoc * - * @param columnIndex the first column is 1, the second is 2, ... - * @param cal the Calendar to use in constructing the date (ignored for Date type) - * @return the column value; if the value is SQL NULL, the value returned is null + * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions + * server-side to avoid client/server timezone inconsistencies. */ override def getDate(columnIndex: Int, cal: Calendar): Date = { getDate(columnIndex) } /** - * Gets the value of the designated column in the current row as a java.sql.Date object. - * The Calendar parameter is ignored for Date type since it is not timezone-aware. + * @inheritdoc * - * @param columnLabel the label for the column specified with the SQL AS clause - * @param cal the Calendar to use in constructing the date (ignored for Date type) - * @return the column value; if the value is SQL NULL, the value returned is null + * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions + * server-side to avoid client/server timezone inconsistencies. */ override def getDate(columnLabel: String, cal: Calendar): Date = getDate(findColumn(columnLabel)) /** - * Gets the value of the designated column in the current row as a java.sql.Time object. - * The Calendar parameter is ignored for Time type since it is not timezone-aware. + * @inheritdoc * - * @param columnIndex the first column is 1, the second is 2, ... - * @param cal the Calendar to use in constructing the time (ignored for Time type) - * @return the column value; if the value is SQL NULL, the value returned is null + * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions + * server-side to avoid client/server timezone inconsistencies. */ override def getTime(columnIndex: Int, cal: Calendar): Time = { getTime(columnIndex) } /** - * Gets the value of the designated column in the current row as a java.sql.Time object. - * The Calendar parameter is ignored for Time type since it is not timezone-aware. + * @inheritdoc * - * @param columnLabel the label for the column specified with the SQL AS clause - * @param cal the Calendar to use in constructing the time (ignored for Time type) - * @return the column value; if the value is SQL NULL, the value returned is null + * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions + * server-side to avoid client/server timezone inconsistencies. */ override def getTime(columnLabel: String, cal: Calendar): Time = getTime(findColumn(columnLabel)) /** + * @inheritdoc + * * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions * server-side to avoid client/server timezone inconsistencies. */ @@ -587,6 +589,8 @@ class SparkConnectResultSet( } /** + * @inheritdoc + * * Note: The Calendar parameter is ignored. Spark Connect handles timezone conversions * server-side to avoid client/server timezone inconsistencies. */ diff --git a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcErrorUtils.scala b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcErrorUtils.scala index 3d9f72d87d150..6480c5d768f3f 100644 --- a/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcErrorUtils.scala +++ b/sql/connect/client/jdbc/src/main/scala/org/apache/spark/sql/connect/client/jdbc/util/JdbcErrorUtils.scala @@ -21,7 +21,7 @@ import java.sql.{Array => _, _} private[jdbc] object JdbcErrorUtils { - def stringfiyTransactionIsolationLevel(level: Int): String = level match { + def stringifyTransactionIsolationLevel(level: Int): String = level match { case Connection.TRANSACTION_NONE => "NONE" case Connection.TRANSACTION_READ_UNCOMMITTED => "READ_UNCOMMITTED" case Connection.TRANSACTION_READ_COMMITTED => "READ_COMMITTED" @@ -31,7 +31,7 @@ private[jdbc] object JdbcErrorUtils { throw new IllegalArgumentException(s"Invalid transaction isolation level: $level") } - def stringfiyHoldability(holdability: Int): String = holdability match { + def stringifyHoldability(holdability: Int): String = holdability match { case ResultSet.HOLD_CURSORS_OVER_COMMIT => "HOLD_CURSORS_OVER_COMMIT" case ResultSet.CLOSE_CURSORS_AT_COMMIT => "CLOSE_CURSORS_AT_COMMIT" case _ => From 41ad6646dc6bad0fe712475cd6959da64e082c0a Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 27 Nov 2025 16:51:07 -0800 Subject: [PATCH 235/400] [SPARK-53614][PYTHON][TESTS][FOLLOW-UP] Fix the maxBytes config in tests ### What changes were proposed in this pull request? Fix the maxBytes config in tests ### Why are the changes needed? the testing configs `(1000, 4096)` were duplicated, IntMax should be tested instead ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53249 from zhengruifeng/test_grouped_conf. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 26ab773478b940d0531d34b71a21758cc5ae6621) Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py index 45b18e9414eb1..fb18c5f062b80 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py @@ -1091,7 +1091,7 @@ def min_max_v(batches: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: .sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 4096), (0, 4096), (1000, 4096)]: + for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { @@ -1137,7 +1137,7 @@ def min_max_v( df.groupby("key").agg(sf.min("v").alias("min"), sf.max("v").alias("max")).sort("key") ).collect() - for maxRecords, maxBytes in [(1000, 4096), (0, 4096), (1000, 4096)]: + for maxRecords, maxBytes in [(1000, 2**31 - 1), (0, 4096), (1000, 4096)]: with self.subTest(maxRecords=maxRecords, maxBytes=maxBytes): with self.sql_conf( { From bad3c84d9b9419950f9c40ee17b457faec5de907 Mon Sep 17 00:00:00 2001 From: huangxiaoping <1754789345@qq.com> Date: Fri, 28 Nov 2025 15:14:30 +0800 Subject: [PATCH 236/400] [SPARK-54299][SQL] Fix the wrong example query in `WindowGroupLimit` ### What changes were proposed in this pull request? Fix the wrong example query in `WindowGroupLimit` ### Why are the changes needed? Fix the incorrect comments ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? + + + + + + + From 7d1bd038161096aa66ee2039b6661f6ee8ccd417 Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Tue, 9 Dec 2025 08:02:21 -0800 Subject: [PATCH 281/400] [SPARK-54637][CONNECT][TESTS] Add SQL API test helpers to SparkConnectServerTest ### What changes were proposed in this pull request? Add testing helpers to SparkConnectServerTest to enable using connect Spark SQL APIs in tests using that helper. ### Why are the changes needed? In Spark 3.5, a testing trait SparkConnectServerTest was introduced that helped test Spark Connect Service with a SparkConnectClient in the same JVM proccess, which tested real Spark Connect code paths (SparkConnectClient communicating with the server over actual connection to the localhost server). Before that, using RemoteSparkSession, server was started in a separate process. It helped * testability: can trigger stuff from the client, then have verification code checking stuff server side. Can also do some more internal server side setup to test specific things. * debugging, as both client and server can be easily connected to by a debugger. At that time, it was impossible to test Spark Connect client SQL APIs (SparkSession, Dataset) this way, because they were in the same namespace as server, and hence couldn't be classloaded together. Since Spark 4.0, there is a new API layer that makes it possible for connect and classic implementation of the interfaces to coexist. With that, testing can be extended to use actual SparkSession and other APIs, instead of having to construct tests using more raw APIs. ### Does this PR introduce _any_ user-facing change? No. It's testing only. ### How was this patch tested? Added SparkConnectServerTestSuite showcasing the new APIs. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Code opus 4.5 Closes #53384 from juliuszsompolski/spark-connect-server-client-test. Authored-by: Juliusz Sompolski Signed-off-by: Dongjoon Hyun (cherry picked from commit 8e75fc96af99b128c7db7abb5518bd57836ca757) Signed-off-by: Dongjoon Hyun --- .../sql/connect/SparkConnectServerTest.scala | 72 +++++- .../connect/SparkConnectServerTestSuite.scala | 207 ++++++++++++++++++ 2 files changed, 278 insertions(+), 1 deletion(-) create mode 100644 sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTestSuite.scala diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala index 1b2b7ab420296..7b9052bb9d2c9 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala @@ -26,14 +26,17 @@ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.connect.proto +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.classic +import org.apache.spark.sql.connect import org.apache.spark.sql.connect.client.{CloseableIterator, CustomSparkConnectBlockingStub, ExecutePlanResponseReattachableIterator, RetryPolicy, SparkConnectClient, SparkConnectStubState} import org.apache.spark.sql.connect.client.arrow.ArrowSerializer import org.apache.spark.sql.connect.common.config.ConnectCommon import org.apache.spark.sql.connect.config.Connect import org.apache.spark.sql.connect.dsl.MockRemoteSession import org.apache.spark.sql.connect.dsl.plans._ -import org.apache.spark.sql.connect.service.{ExecuteHolder, SparkConnectService} +import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionKey, SparkConnectService} import org.apache.spark.sql.test.SharedSparkSession /** @@ -320,4 +323,71 @@ trait SparkConnectServerTest extends SharedSparkSession { val plan = buildPlan(query) runQuery(plan, queryTimeout, iterSleep) } + + /** + * Helper method to create a connect SparkSession that connects to the localhost server. Similar + * to withClient, but provides a full SparkSession API instead of just a client. + * + * @param sessionId + * Optional session ID (defaults to defaultSessionId) + * @param userId + * Optional user ID (defaults to defaultUserId) + * @param f + * Function to execute with the session + */ + protected def withSession(sessionId: String = defaultSessionId, userId: String = defaultUserId)( + f: SparkSession => Unit): Unit = { + withSession(f, sessionId, userId) + } + + /** + * Helper method to create a connect SparkSession with default session and user IDs. + * + * @param f + * Function to execute with the session + */ + protected def withSession(f: SparkSession => Unit): Unit = { + withSession(f, defaultSessionId, defaultUserId) + } + + private def withSession(f: SparkSession => Unit, sessionId: String, userId: String): Unit = { + val client = SparkConnectClient + .builder() + .port(serverPort) + .sessionId(sessionId) + .userId(userId) + .build() + + val session = connect.SparkSession + .builder() + .client(client) + .create() + try f(session) + finally { + session.close() + } + } + + /** + * Get the server-side SparkSession corresponding to a client SparkSession. + * + * This helper takes a sql.SparkSession (which is assumed to be a connect.SparkSession), + * extracts the userId and sessionId from it, and looks up the corresponding server-side classic + * SparkSession using SparkConnectSessionManager. + * + * @param clientSession + * The client SparkSession (must be a connect.SparkSession) + * @return + * The server-side classic SparkSession + */ + protected def getServerSession(clientSession: SparkSession): classic.SparkSession = { + val connectSession = clientSession.asInstanceOf[connect.SparkSession] + val userId = connectSession.client.userId + val sessionId = connectSession.sessionId + val key = SessionKey(userId, sessionId) + SparkConnectService.sessionManager + .getIsolatedSessionIfPresent(key) + .get + .session + } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTestSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTestSuite.scala new file mode 100644 index 0000000000000..c14114ced6634 --- /dev/null +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTestSuite.scala @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect + +import org.scalatest.time.SpanSugar._ + +/** + * Test suite showcasing the APIs provided by SparkConnectServerTest trait. + * + * This suite demonstrates: + * - Session and client helper methods (withSession, withClient, getServerSession) + * - Low-level stub helpers (withRawBlockingStub, withCustomBlockingStub) + * - Plan building helpers (buildPlan, buildExecutePlanRequest, etc.) + * - Assertion helpers for execution state + */ +class SparkConnectServerTestSuite extends SparkConnectServerTest { + + test("withSession: execute SQL and collect results") { + withSession { session => + val df = session.sql("SELECT 1 as value") + val result = df.collect() + assert(result.length == 1) + assert(result(0).getInt(0) == 1) + } + } + + test("withSession: with custom session and user IDs") { + val customSessionId = java.util.UUID.randomUUID().toString + val customUserId = "test-user" + withSession(sessionId = customSessionId, userId = customUserId) { session => + val df = session.sql("SELECT 'hello' as greeting") + val result = df.collect() + assert(result.length == 1) + assert(result(0).getString(0) == "hello") + } + } + + test("withSession: DataFrame operations") { + withSession { session => + val df = session.range(10) + assert(df.count() == 10) + + val sum = df.selectExpr("sum(id)").collect()(0).getLong(0) + assert(sum == 45) // 0 + 1 + ... + 9 = 45 + } + } + + test("withClient: execute plan and iterate results") { + withClient { client => + val plan = buildPlan("SELECT 1 as x, 2 as y") + val iter = client.execute(plan) + var hasResults = false + while (iter.hasNext) { + iter.next() + hasResults = true + } + assert(hasResults) + } + } + + test("withClient: with custom session and user IDs") { + val customSessionId = java.util.UUID.randomUUID().toString + val customUserId = "custom-user" + withClient(sessionId = customSessionId, userId = customUserId) { client => + val plan = buildPlan("SELECT 42") + val iter = client.execute(plan) + while (iter.hasNext) iter.next() + } + } + + test("getServerSession: returns server-side classic session") { + withSession { clientSession => + clientSession.sql("SELECT 1").collect() + + val serverSession = getServerSession(clientSession) + + assert(serverSession != null) + assert(serverSession.sparkContext != null) + } + } + + test("getServerSession: client and server share configuration") { + withSession { clientSession => + clientSession.sql("SET spark.sql.shuffle.partitions=17").collect() + + val serverSession = getServerSession(clientSession) + assert(serverSession.conf.get("spark.sql.shuffle.partitions") == "17") + } + } + + test("getServerSession: register and use temporary view from server") { + withSession { clientSession => + clientSession.sql("SELECT 1 as a, 2 as b").collect() + + val serverSession = getServerSession(clientSession) + + // Create a temp view on the server side + import serverSession.implicits._ + val serverDf = Seq((100, "server"), (200, "side")).toDF("num", "source") + serverDf.createOrReplaceTempView("server_view") + + // Access the view from the client + val result = clientSession.sql("SELECT * FROM server_view ORDER BY num").collect() + assert(result.length == 2) + assert(result(0).getInt(0) == 100) + assert(result(0).getString(1) == "server") + assert(result(1).getInt(0) == 200) + assert(result(1).getString(1) == "side") + } + } + + test("withRawBlockingStub: execute plan via raw gRPC stub") { + withRawBlockingStub { stub => + val request = buildExecutePlanRequest(buildPlan("SELECT 'raw' as mode")) + val iter = stub.executePlan(request) + assert(iter.hasNext) + while (iter.hasNext) iter.next() + } + } + + test("withCustomBlockingStub: execute plan via custom blocking stub") { + withCustomBlockingStub() { stub => + val request = buildExecutePlanRequest(buildPlan("SELECT 'custom' as mode")) + val iter = stub.executePlan(request) + while (iter.hasNext) iter.next() + } + } + + test("buildPlan: creates plan from SQL query") { + val plan = buildPlan("SELECT 1, 2, 3") + assert(plan.hasRoot) + } + + test("buildSqlCommandPlan: creates command plan") { + val plan = buildSqlCommandPlan("SET spark.sql.adaptive.enabled=true") + assert(plan.hasCommand) + assert(plan.getCommand.hasSqlCommand) + } + + test("buildLocalRelation: creates plan from local data") { + val data = Seq((1, "a"), (2, "b"), (3, "c")) + val plan = buildLocalRelation(data) + assert(plan.hasRoot) + assert(plan.getRoot.hasLocalRelation) + } + + test("buildExecutePlanRequest: creates request with options") { + val plan = buildPlan("SELECT 1") + val request = buildExecutePlanRequest(plan) + assert(request.hasPlan) + assert(request.hasUserContext) + assert(request.getSessionId == defaultSessionId) + } + + test("buildExecutePlanRequest: with custom session and operation IDs") { + val plan = buildPlan("SELECT 1") + val customSessionId = "my-session" + val customOperationId = "my-operation" + val request = + buildExecutePlanRequest(plan, sessionId = customSessionId, operationId = customOperationId) + assert(request.getSessionId == customSessionId) + assert(request.getOperationId == customOperationId) + } + + test("runQuery: executes query string with timeout") { + runQuery("SELECT * FROM range(100)", 30.seconds) + } + + test("runQuery: executes plan with timeout and iter sleep") { + val plan = buildPlan("SELECT * FROM range(10)") + runQuery(plan, 30.seconds, iterSleep = 10) + } + + test("assertNoActiveExecutions: verifies clean state") { + assertNoActiveExecutions() + } + + test("assertNoActiveRpcs: verifies no active RPCs") { + assertNoActiveRpcs() + } + + test("eventuallyGetExecutionHolder: retrieves active execution") { + withRawBlockingStub { stub => + val request = buildExecutePlanRequest(buildPlan("SELECT * FROM range(1000000)")) + val iter = stub.executePlan(request) + iter.hasNext // trigger execution + + val holder = eventuallyGetExecutionHolder + assert(holder != null) + assert(holder.operationId == request.getOperationId) + } + } +} From 374d5080f68f746306515820f9b9790dfb5a2baf Mon Sep 17 00:00:00 2001 From: kepler62f Date: Wed, 10 Dec 2025 09:04:39 -0800 Subject: [PATCH 282/400] [MINOR][DOCS] Fix a typo in `spark-standalone.md` ### What changes were proposed in this pull request? A minor typo fix to documentation. ### Why are the changes needed? Clearer documentation for users. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53285 from kepler62f/master. Authored-by: kepler62f Signed-off-by: Dongjoon Hyun (cherry picked from commit a20ecc9a28fbed5db7832f7c62324b02d7b1ef65) Signed-off-by: Dongjoon Hyun --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 5a8eb3f1e0602..ec1656b0348c8 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -389,7 +389,7 @@ SPARK_MASTER_OPTS supports the following system properties: spark.dead.worker.persistence 15 - Number of iterations to keep the deae worker information in UI. By default, the dead worker is visible for (15 + 1) * spark.worker.timeout since its last heartbeat. + Number of iterations to keep the dead worker information in UI. By default, the dead worker is visible for (15 + 1) * spark.worker.timeout since its last heartbeat. 0.8.0 From c27faeb283cea190185b5d86e415024575fa72a9 Mon Sep 17 00:00:00 2001 From: Jerry Peng Date: Wed, 10 Dec 2025 09:10:40 -0800 Subject: [PATCH 283/400] [SPARK-53998][TESTS] Add addition E2E tests for RTM ### What changes were proposed in this pull request? Add some additional end to end tests for RTM ### Why are the changes needed? To have better test coverage for RTM functionality ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? N/A. Only tests are added ### Was this patch authored or co-authored using generative AI tooling? no Closes #52870 from jerrypeng/SPARK-53998-2. Authored-by: Jerry Peng Signed-off-by: Dongjoon Hyun (cherry picked from commit 7df7dad29a45f447ed57ba190ecedd3f0feaec17) Signed-off-by: Dongjoon Hyun --- .../StreamRealTimeModeE2ESuite.scala | 393 ++++++++++++++++++ .../StreamRealTimeModeSuiteBase.scala | 6 + 2 files changed, 399 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeE2ESuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeE2ESuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeE2ESuite.scala new file mode 100644 index 0000000000000..3615edc75cb2c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeE2ESuite.scala @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.mutable + +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkContext +import org.apache.spark.sql.{ForeachWriter, Row} +import org.apache.spark.sql.execution.datasources.v2.LowLatencyClock +import org.apache.spark.sql.execution.streaming.LowLatencyMemoryStream +import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryWrapper +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.streaming.util.GlobalSingletonManualClock +import org.apache.spark.sql.test.TestSparkSession +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + +class StreamRealTimeModeE2ESuite extends StreamRealTimeModeE2ESuiteBase { + + import testImplicits._ + + override protected def createSparkSession = + new TestSparkSession( + new SparkContext( + "local[15]", + "streaming-rtm-e2e-context", + sparkConf.set("spark.sql.shuffle.partitions", "5") + ) + ) + + private def runForeachTest(withUnion: Boolean): Unit = { + var query: StreamingQuery = null + try { + withTempDir { checkpointDir => + val clock = new GlobalSingletonManualClock() + LowLatencyClock.setClock(clock) + val uniqueSinkName = if (withUnion) { + sinkName + "-union" + } else { + sinkName + } + + val read = LowLatencyMemoryStream[(String, Int)](5) + val read1 = LowLatencyMemoryStream[(String, Int)](5) + val dataframe = if (withUnion) { + read.toDF().union(read1.toDF()) + } else { + read.toDF() + } + + query = dataframe + .select(col("_1").as("key"), col("_2").as("value")) + .select( + concat( + col("key").cast("STRING"), + lit("-"), + col("value").cast("STRING") + ).as("output") + ) + .writeStream + .outputMode(OutputMode.Update()) + .foreach(new ForeachWriter[Row] { + private var batchPartitionId: String = null + private val processedThisBatch = new ConcurrentLinkedQueue[String]() + override def open(partitionId: Long, epochId: Long): Boolean = { + ResultsCollector + .computeIfAbsent(uniqueSinkName, (_) => new ConcurrentLinkedQueue[String]()) + batchPartitionId = s"$uniqueSinkName-$epochId-$partitionId" + assert( + !ResultsCollector.containsKey(batchPartitionId), + s"should NOT contain batchPartitionId ${batchPartitionId}" + ) + ResultsCollector + .put(batchPartitionId, new ConcurrentLinkedQueue[String]()) + true + } + + override def process(value: Row): Unit = { + val v = value.getAs[String]("output") + ResultsCollector.get(uniqueSinkName).add(v) + processedThisBatch.add(v) + } + + override def close(errorOrNull: Throwable): Unit = { + + assert( + ResultsCollector.containsKey(batchPartitionId), + s"should contain batchPartitionId ${batchPartitionId}" + ) + ResultsCollector.get(batchPartitionId).addAll(processedThisBatch) + processedThisBatch.clear() + } + }) + .option("checkpointLocation", checkpointDir.getName) + .queryName("foreach") + // doesn't matter the batch duration set here since we are going + // to manually control batch durations via manual clock + .trigger(defaultTrigger) + .start() + + val expectedResults = mutable.ListBuffer[String]() + val expectedResultsByBatch = mutable.HashMap[Int, mutable.ListBuffer[String]]() + + val numRows = 10 + for (i <- 0 until 3) { + expectedResultsByBatch(i) = new mutable.ListBuffer[String]() + for (key <- List("a", "b", "c")) { + for (j <- 1 to numRows) { + read.addData((key, 1)) + val data = s"$key-1" + expectedResults += data + expectedResultsByBatch(i) += data + } + } + + if (withUnion) { + for (key <- List("d", "e", "f")) { + for (j <- 1 to numRows) { + read1.addData((key, 2)) + val data = s"$key-2" + expectedResults += data + expectedResultsByBatch(i) += data + } + } + } + + eventually(timeout(60.seconds)) { + ResultsCollector + .get(uniqueSinkName) + .toArray(new Array[String](ResultsCollector.get(uniqueSinkName).size())) + .toList + .sorted should equal(expectedResults.sorted) + } + + clock.advance(defaultTrigger.batchDurationMs) + eventually(timeout(60.seconds)) { + query + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + .getLatestExecutionContext() + .batchId should be(i + 1) + query.lastProgress.sources(0).numInputRows should be(numRows * 3) + + val commitedResults = new mutable.ListBuffer[String]() + val numPartitions = if (withUnion) 10 else 5 + for (v <- 0 until numPartitions) { + val it = ResultsCollector.get(s"$uniqueSinkName-${i}-$v").iterator() + while (it.hasNext) { + commitedResults += it.next() + } + } + + commitedResults.sorted should equal(expectedResultsByBatch(i).sorted) + } + } + } + } finally { + if (query != null) { + query.stop() + } + } + } + + private def runMapPartitionsTest(withUnion: Boolean): Unit = { + var query: StreamingQuery = null + try { + withTempDir { checkpointDir => + val clock = new GlobalSingletonManualClock() + LowLatencyClock.setClock(clock) + val uniqueSinkName = if (withUnion) { + sinkName + "mapPartitions-union" + } else { + sinkName + "mapPartitions" + } + + val read = LowLatencyMemoryStream[(String, Int)](5) + val read1 = LowLatencyMemoryStream[(String, Int)](5) + val dataframe = if (withUnion) { + read.toDF().union(read1.toDF()) + } else { + read.toDF() + } + + val df = dataframe + .select(col("_1").as("key"), col("_2").as("value")) + .select( + concat( + col("key").cast("STRING"), + lit("-"), + col("value").cast("STRING") + ).as("output") + ) + .as[String] + .mapPartitions(rows => { + rows.map(row => { + val collector = ResultsCollector + .computeIfAbsent(uniqueSinkName, (_) => new ConcurrentLinkedQueue[String]()) + collector.add(row) + row + }) + }) + .toDF() + + query = runStreamingQuery(sinkName, df) + + val expectedResults = mutable.ListBuffer[String]() + val expectedResultsByBatch = mutable.HashMap[Int, mutable.ListBuffer[String]]() + + val numRows = 10 + for (i <- 0 until 3) { + expectedResultsByBatch(i) = new mutable.ListBuffer[String]() + for (key <- List("a", "b", "c")) { + for (j <- 1 to numRows) { + read.addData((key, 1)) + val data = s"$key-1" + expectedResults += data + expectedResultsByBatch(i) += data + } + } + + if (withUnion) { + for (key <- List("d", "e", "f")) { + for (j <- 1 to numRows) { + read1.addData((key, 2)) + val data = s"$key-2" + expectedResults += data + expectedResultsByBatch(i) += data + } + } + } + + // results collected from mapPartitions + eventually(timeout(60.seconds)) { + ResultsCollector + .get(uniqueSinkName) + .toArray(new Array[String](ResultsCollector.get(uniqueSinkName).size())) + .toList + .sorted should equal(expectedResults.sorted) + } + + // results collected from foreach sink + eventually(timeout(60.seconds)) { + ResultsCollector + .get(sinkName) + .toArray(new Array[String](ResultsCollector.get(sinkName).size())) + .toList + .sorted should equal(expectedResults.sorted) + } + + clock.advance(defaultTrigger.batchDurationMs) + eventually(timeout(60.seconds)) { + query + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + .getLatestExecutionContext() + .batchId should be(i + 1) + query.lastProgress.sources(0).numInputRows should be(numRows * 3) + } + } + } + } finally { + if (query != null) { + query.stop() + } + } + } + + test("foreach") { + runForeachTest(withUnion = false) + } + + test("union - foreach") { + runForeachTest(withUnion = true) + } + + test("mapPartitions") { + runMapPartitionsTest(withUnion = false) + } + + test("union - mapPartitions") { + runMapPartitionsTest(withUnion = true) + } + + test("scala stateless UDF") { + val myUDF = (id: Int) => id + 1 + val udf = spark.udf.register("myUDF", myUDF) + val (read, clock) = createMemoryStream() + + val df = read + .toDF() + .select(col("_1").as("key"), udf(col("_2")).as("value_plus_1")) + .select(concat(col("key"), lit("-"), col("value_plus_1").cast("STRING")).as("output")) + + var query: StreamingQuery = null + try { + query = runStreamingQuery("scala_udf", df) + processBatches(query, read, clock, 10, 3, (key, value) => Array(s"$key-${value + 1}")) + } finally { + if (query != null) query.stop() + } + } + + test("stream static join") { + val (read, clock) = createMemoryStream() + val staticDf = spark + .range(1, 31, 1, 10) + .selectExpr("id AS join_key", "id AS join_value") + // This will produce HashAggregateExec which should not be blocked by allowList + // since it's the batch subquery + .groupBy("join_key") + .agg(max($"join_value").as("join_value")) + + val df = read + .toDF() + .select(col("_1").as("key"), col("_2").as("value")) + .join(staticDf, col("value") === col("join_key")) + .select(concat(col("key"), lit("-"), col("value"), lit("-"), col("join_value")).as("output")) + + var query: StreamingQuery = null + try { + query = runStreamingQuery("stream_static_join", df) + processBatches(query, read, clock, 10, 3, (key, value) => Array(s"$key-$value-$value")) + } finally { + if (query != null) query.stop() + } + } + + test("to_json and from_json round-trip") { + val (read, clock) = createMemoryStream() + val schema = new StructType().add("key", StringType).add("value", IntegerType) + + val df = read + .toDF() + .select(struct(col("_1").as("key"), col("_2").as("value")).as("json")) + .select(from_json(to_json(col("json")), schema).as("json")) + .select(concat(col("json.key"), lit("-"), col("json.value"))) + + var query: StreamingQuery = null + try { + query = runStreamingQuery("json_roundtrip", df) + processBatches(query, read, clock, 10, 3, (key, value) => Array(s"$key-$value")) + } finally { + if (query != null) query.stop() + } + } + + test("generateExec passthrough") { + val (read, clock) = createMemoryStream() + + val df = read + .toDF() + .select(col("_1").as("key"), col("_2").as("value")) + .withColumn("value_array", array(col("value"), -col("value"))) + df.createOrReplaceTempView("tempView") + val explodeDF = + spark + .sql("select key, explode(value_array) as exploded_value from tempView") + .select(concat(col("key"), lit("-"), col("exploded_value").cast("STRING")).as("output")) + + var query: StreamingQuery = null + try { + query = runStreamingQuery("generateExec_passthrough", explodeDF) + processBatches( + query, + read, + clock, + 10, + 3, + (key, value) => Array(s"$key-$value", s"$key--$value") + ) + } finally { + if (query != null) query.stop() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeSuiteBase.scala index 9199580f65872..7ec5d8e51f098 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamRealTimeModeSuiteBase.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryWrapper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.GlobalSingletonManualClock import org.apache.spark.sql.test.TestSparkSession +import org.apache.spark.util.SystemClock /** * Base class for tests that require real-time mode. @@ -45,6 +46,11 @@ trait StreamRealTimeModeSuiteBase extends StreamTest with Matchers { defaultTrigger.batchDurationMs) } + override def beforeAll(): Unit = { + super.beforeAll() + LowLatencyClock.setClock(new SystemClock) + } + override protected def createSparkSession = new TestSparkSession( new SparkContext( "local[10]", // Ensure enough number of cores to ensure concurrent schedule of all tasks. From 9dcf671e8d32cc54ea7ebd2ef4e148a7628b68dc Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 10 Dec 2025 09:15:23 -0800 Subject: [PATCH 284/400] [SPARK-54340][PYTHON][FOLLOW-UP] Add link and examples for `run-with-viztracer` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add link and examples for `run-with-viztracer` ### Why are the changes needed? to make it easier for developers to have a try ### Does this PR introduce _any_ user-facing change? dev-only changes ### How was this patch tested? manually check ```sh (spark_dev_313) ➜ spark git:(doc_viz) python/run-with-viztracer -h Usage: run-with-viztracer your_original_commands To view the profiling results, run: vizviewer pyspark_*.json Environment: If SPARK_VIZTRACER_OUTPUT_DIR is set, the output will be saved to the directory. Otherwise, it will be saved to the current directory. Requirements: - viztracer must be installed (pip install viztracer) Check the following documentation for more information on using viztracer: https://viztracer.readthedocs.io/en/latest/ Examples: - Start pyspark shell python/run-with-viztracer bin/pyspark --conf spark.driver.memory=16g - Start pyspark shell in Connect mode python/run-with-viztracer bin/pyspark --remote local ``` ### Was this patch authored or co-authored using generative AI tooling? no Closes #53418 from zhengruifeng/doc_viz. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun (cherry picked from commit 4520c17b19f6d90ef393d290a452faf8865e6e7b) Signed-off-by: Dongjoon Hyun --- python/run-with-viztracer | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/run-with-viztracer b/python/run-with-viztracer index 448b8e20b9c1d..0f7a4d8494f0e 100755 --- a/python/run-with-viztracer +++ b/python/run-with-viztracer @@ -37,6 +37,17 @@ Environment: Requirements: - viztracer must be installed (pip install viztracer) + +Check the following documentation for more information on using viztracer: + + https://viztracer.readthedocs.io/en/latest/ + +Examples: + - Start pyspark shell + $0 bin/pyspark --conf spark.driver.memory=16g + + - Start pyspark shell in Connect mode + $0 bin/pyspark --remote local EOF exit 0 } From 10473209949065063c8d7c5fcf0754b1a41172d4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Dec 2025 13:40:18 -0800 Subject: [PATCH 285/400] [SPARK-54656][SQL] Refactor SupportsPushDownVariants to be a ScanBuilder mix-in SupportsPushDownVariants was a Scan mix-in in #52578. This patch changes it to be a ScanBuilder mix-in to follow the established patterns in the codebase. SupportsPushDownVariants was a Scan mix-in in #52578. This patch changes it to be a ScanBuilder mix-in to follow the established patterns in the codebase, e.g, join pushdown, aggregate pushdown...etc. No Unit tests Generated-by: Claude Code v2.0.14 Closes #53276 from viirya/pushvariantdsv2-pr-refactor. Lead-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun (cherry picked from commit 0ba4a30d52410de550f674ad2a5ab4eba478b132) Signed-off-by: Dongjoon Hyun --- .../SupportsPushDownVariantExtractions.java | 63 ++++++ .../read/SupportsPushDownVariants.java | 77 ------- .../sql/connector/read/VariantAccessInfo.java | 105 ---------- .../sql/connector/read/VariantExtraction.java | 64 ++++++ .../connector/VariantExtractionImpl.scala | 45 +++++ .../datasources/PushVariantIntoScan.scala | 116 +---------- .../v2/V2ScanRelationPushDown.scala | 191 +++++++++++++++++- .../datasources/v2/parquet/ParquetScan.scala | 110 +++++----- .../v2/parquet/ParquetScanBuilder.scala | 17 +- 9 files changed, 440 insertions(+), 348 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariantExtractions.java delete mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariants.java delete mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantAccessInfo.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/VariantExtraction.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/VariantExtractionImpl.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariantExtractions.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariantExtractions.java new file mode 100644 index 0000000000000..750e0479e542d --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownVariantExtractions.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.read; + +import org.apache.spark.annotation.Experimental; + +/** + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to + * support pushing down variant field extraction operations to the data source. + *

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/dev/test-classes.txt b/dev/test-classes.txt index e69de29bb2d1d..2dc6b290ad4f9 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -0,0 +1,7 @@ +sql/core/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/IntSumUdf.class +sql/core/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/HelloWithPackage.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index e69de29bb2d1d..bd8fc93bc9f0f 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -0,0 +1,17 @@ +core/src/test/resources/TestHelloV2_2.13.jar +core/src/test/resources/TestHelloV3_2.13.jar +core/src/test/resources/TestUDTF.jar +data/artifact-tests/junitLargeJar.jar +data/artifact-tests/smallJar.jar +sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar +sql/connect/client/jvm/src/test/resources/udf2.13.jar +sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar +sql/connect/common/src/test/resources/artifact-tests/smallJar.jar +sql/core/src/test/resources/SPARK-33084.jar +sql/core/src/test/resources/artifact-tests/udf_noA.jar +sql/hive-thriftserver/src/test/resources/TestUDTF.jar +sql/hive/src/test/noclasspath/hive-test-udfs.jar +sql/hive/src/test/resources/SPARK-21101-1.0.jar +sql/hive/src/test/resources/TestUDTF.jar +sql/hive/src/test/resources/data/files/TestSerDe.jar +sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..c89830f127c0c18690cec93ec229c6d3b50f7e95 GIT binary patch literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class new file mode 100644 index 0000000000000000000000000000000000000000..56725764de20539b9c16799d2bff4ee3a4e7de97 GIT binary patch literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..6da55d8b8520dcc03c250a605151cc0d23a45518 GIT binary patch literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar deleted file mode 100644 index 0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q From 2808b96a1b8d0dec5075967c50b3590753054225 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Mon, 24 Nov 2025 01:09:50 +0000 Subject: [PATCH 207/400] Preparing Spark release v4.1.0-rc1 --- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 49 files changed, 50 insertions(+), 50 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 8de2333671abf..743f74753b1f8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 5d923ecc69ffe..1afbf81679906 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index f8283b5ddb07c..167582598a920 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 60ad971573997..8cede6cbf288e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index c4451923b17a5..5e6348a717735 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8cf59f603a5bb..161802d43ca89 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index de93a9205ac91..5e851791a6975 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 47fc39abd2a67..869931e403349 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index ba3603f810856..dbac5e4552df5 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index df3bc5adb10bd..1805f8462c278 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index 0fe977b8eadd2..adecb4b4185d3 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index 9c3f2249e2f19..bad1f7529c054 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index 09fabdf64dba8..b5af1a093be36 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 0305e8895ba3c..0022adff1a208 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index 4ab99f7929591..b482aeb77a96d 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 57840a8a8d6b3..5da0ee078e26e 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index 62b0807e959d8..2479590341822 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 4903adfe420fd..9acc9b65d7653 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index aa90e3c8a94f5..54d5747e78aef 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 0eee0123ec071..9d486a868c3de 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 845a7bb64cd35..a78b722eb6bf0 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 4485b1ea414df..7ee7e096d38c8 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ef408a763323a..2281f3e197f88 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 2cb5b0704bae6..dc3890e4522ee 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,7 +19,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.0-SNAPSHOT +SPARK_VERSION: 4.1.0 SPARK_VERSION_SHORT: 4.1.0 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:latest"] + 'facetFilters': ["version:4.1.0"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index 5b654b89d7fd0..cd17565a1d98b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c165485652861..602817ca792e4 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index dbdbe8846e189..fc593ff2c6196 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index a4a44db2e6ac6..e49a0678d7182 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index d0d310b9371df..6695e77228935 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 0d87640c6b47d..b31c2b7fb3a24 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index b0a47082d0770..1824a28614bd3 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 374bc8bbd8a47..9b0e7c1f5ce40 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.0.dev0" +__version__: str = "4.1.0" diff --git a/repl/pom.xml b/repl/pom.xml index 7b515eacc5401..53bc41d397d60 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 972b618a5caa2..c5f2749ea9fb0 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index d470356f39e66..3b471a8ff4133 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 0629c24c56dc4..6d0d0305ef710 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 184d39c4b8ea1..e7997cf61a457 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6dcedfa6408e3..8eef00489669d 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index af34a65f9bdec..c6a5805bb0f5c 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 9514ed1ff202f..6e99700c90c51 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 4b7df7a232d34..3b74dca497387 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index 547797a1a47b3..ab19897f64285 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 37e565bf25872..febf7a2beddf4 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index b3e0f947296f7..e3aa8f1f3bb4c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 135b84cd01f85..8c4087372913a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1d4c7b784c0af..e51872a36ca5b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index cdf3daa4329a6..6e4a4d64f7930 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index 8128a44a4fd1b..9602aa01e7c4d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 6872b5803934d..cfc6a7150d858 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0-SNAPSHOT + 4.1.0 ../pom.xml From eb9b09fb51fc7c2f8f3df044972d4e7943357c4a Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Mon, 24 Nov 2025 01:09:50 +0000 Subject: [PATCH 208/400] Revert "Removing test jars and class files" This reverts commit 8b6a2ff9ca54f1639790e178225af9b534db7348. --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 0 -> 3827 bytes core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes data/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes data/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes dev/test-classes.txt | 7 +++++++ dev/test-jars.txt | 17 +++++++++++++++++ .../src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../artifact-tests/smallClassFileDup.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 0 -> 6119 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../artifact-tests/HelloWithPackage.class | Bin 0 -> 635 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 0 -> 1333 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 0 -> 5545 bytes .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 0 -> 35660 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 0 -> 7439 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../test/resources/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 0 -> 19579 bytes 26 files changed, 24 insertions(+) create mode 100644 core/src/test/resources/TestHelloV2_2.13.jar create mode 100644 core/src/test/resources/TestHelloV3_2.13.jar create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100755 data/artifact-tests/junitLargeJar.jar create mode 100755 data/artifact-tests/smallJar.jar create mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar create mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar create mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar create mode 100644 sql/core/src/test/resources/SPARK-33084.jar create mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class create mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class create mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class create mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class create mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar create mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar create mode 100644 sql/hive/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar create mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0c292e7d81ad741dbc11177224b9c30519b7a362 GIT binary patch literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/dev/test-classes.txt b/dev/test-classes.txt index 2dc6b290ad4f9..e69de29bb2d1d 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -1,7 +0,0 @@ -sql/core/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/IntSumUdf.class -sql/core/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/HelloWithPackage.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index bd8fc93bc9f0f..e69de29bb2d1d 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -1,17 +0,0 @@ -core/src/test/resources/TestHelloV2_2.13.jar -core/src/test/resources/TestHelloV3_2.13.jar -core/src/test/resources/TestUDTF.jar -data/artifact-tests/junitLargeJar.jar -data/artifact-tests/smallJar.jar -sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar -sql/connect/client/jvm/src/test/resources/udf2.13.jar -sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar -sql/connect/common/src/test/resources/artifact-tests/smallJar.jar -sql/core/src/test/resources/SPARK-33084.jar -sql/core/src/test/resources/artifact-tests/udf_noA.jar -sql/hive-thriftserver/src/test/resources/TestUDTF.jar -sql/hive/src/test/noclasspath/hive-test-udfs.jar -sql/hive/src/test/resources/SPARK-21101-1.0.jar -sql/hive/src/test/resources/TestUDTF.jar -sql/hive/src/test/resources/data/files/TestSerDe.jar -sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar deleted file mode 100644 index c89830f127c0c18690cec93ec229c6d3b50f7e95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class deleted file mode 100644 index 56725764de20539b9c16799d2bff4ee3a4e7de97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar deleted file mode 100755 index 6da55d8b8520dcc03c250a605151cc0d23a45518..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

""" + print("Enabling TIME data type") + jspark.sql("SET spark.sql.timeType.enabled = true") print("Running SQL examples to generate formatted output.") for key, infos in _list_grouped_function_infos(jvm): examples = _make_pretty_examples(jspark, infos) From 33ceb0daf9b434ac5fb32e582c7645200ac5c8b6 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Thu, 11 Dec 2025 14:00:23 +0000 Subject: [PATCH 290/400] Removing test jars and class files --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 3827 -> 0 bytes core/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes data/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes data/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes dev/test-classes.txt | 7 ------- dev/test-jars.txt | 17 ----------------- .../src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 5674 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../artifact-tests/smallClassFileDup.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 6119 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../artifact-tests/HelloWithPackage.class | Bin 635 -> 0 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 1333 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 5545 -> 0 bytes .../src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 35660 -> 0 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 7439 -> 0 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../test/resources/data/files/TestSerDe.jar | Bin 3838 -> 0 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 19579 -> 0 bytes 26 files changed, 24 deletions(-) delete mode 100644 core/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 core/src/test/resources/TestHelloV3_2.13.jar delete mode 100644 core/src/test/resources/TestUDTF.jar delete mode 100755 data/artifact-tests/junitLargeJar.jar delete mode 100755 data/artifact-tests/smallJar.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar delete mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar delete mode 100644 sql/core/src/test/resources/SPARK-33084.jar delete mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class delete mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class delete mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class delete mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class delete mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar delete mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar delete mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar delete mode 100644 sql/hive/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar delete mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar deleted file mode 100644 index 0c292e7d81ad741dbc11177224b9c30519b7a362..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar deleted file mode 100644 index 514f2d5d26fd358ad5647e0e75edb8ce77b69e30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar deleted file mode 100755 index 3c4930e8e9549d17b89cd3ecb028f048f331797c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar deleted file mode 100644 index 61e1663ad3a2804eb7c39960ea51e56a0ebfaf94..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class deleted file mode 100644 index f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class deleted file mode 100644 index 75a41446cfca1f93570ab90a74d80b51e0fb59e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar deleted file mode 100644 index 768b2334db5c3aa8b1e4186af5fde86120d3231b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..3c4930e8e9549d17b89cd3ecb028f048f331797c GIT binary patch literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar new file mode 100644 index 0000000000000000000000000000000000000000..61e1663ad3a2804eb7c39960ea51e56a0ebfaf94 GIT binary patch literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class new file mode 100644 index 0000000000000000000000000000000000000000..f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce GIT binary patch literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class new file mode 100644 index 0000000000000000000000000000000000000000..75a41446cfca1f93570ab90a74d80b51e0fb59e4 GIT binary patch literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar new file mode 100644 index 0000000000000000000000000000000000000000..768b2334db5c3aa8b1e4186af5fde86120d3231b GIT binary patch literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar deleted file mode 100755 index 3c4930e8e9549d17b89cd3ecb028f048f331797c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar deleted file mode 100644 index 61e1663ad3a2804eb7c39960ea51e56a0ebfaf94..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class deleted file mode 100644 index f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class deleted file mode 100644 index 75a41446cfca1f93570ab90a74d80b51e0fb59e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar deleted file mode 100644 index 768b2334db5c3aa8b1e4186af5fde86120d3231b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..3c4930e8e9549d17b89cd3ecb028f048f331797c GIT binary patch literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar new file mode 100644 index 0000000000000000000000000000000000000000..61e1663ad3a2804eb7c39960ea51e56a0ebfaf94 GIT binary patch literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class new file mode 100644 index 0000000000000000000000000000000000000000..f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce GIT binary patch literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class new file mode 100644 index 0000000000000000000000000000000000000000..75a41446cfca1f93570ab90a74d80b51e0fb59e4 GIT binary patch literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar new file mode 100644 index 0000000000000000000000000000000000000000..768b2334db5c3aa8b1e4186af5fde86120d3231b GIT binary patch literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar deleted file mode 100755 index 3c4930e8e9549d17b89cd3ecb028f048f331797c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar deleted file mode 100644 index 61e1663ad3a2804eb7c39960ea51e56a0ebfaf94..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class deleted file mode 100644 index f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class deleted file mode 100644 index 75a41446cfca1f93570ab90a74d80b51e0fb59e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar deleted file mode 100644 index 768b2334db5c3aa8b1e4186af5fde86120d3231b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..3c4930e8e9549d17b89cd3ecb028f048f331797c GIT binary patch literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar new file mode 100644 index 0000000000000000000000000000000000000000..61e1663ad3a2804eb7c39960ea51e56a0ebfaf94 GIT binary patch literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class new file mode 100644 index 0000000000000000000000000000000000000000..f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce GIT binary patch literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class new file mode 100644 index 0000000000000000000000000000000000000000..75a41446cfca1f93570ab90a74d80b51e0fb59e4 GIT binary patch literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar new file mode 100644 index 0000000000000000000000000000000000000000..768b2334db5c3aa8b1e4186af5fde86120d3231b GIT binary patch literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar deleted file mode 100755 index 3c4930e8e9549d17b89cd3ecb028f048f331797c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar deleted file mode 100644 index 61e1663ad3a2804eb7c39960ea51e56a0ebfaf94..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class deleted file mode 100644 index f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class deleted file mode 100644 index 75a41446cfca1f93570ab90a74d80b51e0fb59e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class deleted file mode 100755 index e796030e471b058b37fc83f213dc20ad3c034d9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar deleted file mode 100644 index 768b2334db5c3aa8b1e4186af5fde86120d3231b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

`)ZZo?PM~)f*OI;Nlt*N`a@~;G*&9Se} zx7IYsjdZR<6_gi5kW2KVp}0^P9*rSaDB1##h|&0#r34|giGc9hbX|^muucL7! zo~5+-$|pH`Kl`6XgXw z{Z4oG&s;&`N_onhVy(rgy&PvqMRqE~TAbOq$XuB{jB#FBS$&RD9*HcaX$@M;r51X^ z&RI%ZYjhXgB5qlvl9@%Z3i+gC)o}?dm%e{+K5oGJJPO~xe6jz>41>Snd>sCl`}!}y z_ixYW&*i3`PdUhn$V|V|%>0Ws`C>R4@|EyaAsLu*Zc;NbOr>DvIQhn;{tB*G@Tel4cs0x*o{#-Z0yi`T{B3+-uIs z{dWnGDy3L3q=!|#oCGj|2N5CeF2x6T(J)4WOi7D2y+vA?Q3_`Ch_Vg%PuX>8@7d;0 zVt-Iu_rP5XpL9=Y9I*~uKR{Fe7tj|CNEL1zfPc(zK`=_xkX0N7Quy)@=L=Z>rj|Gj z-j}YsoC#ev^)@Bz3E0A8m~fIND=R|V9-(0o4O7m+nS->&uL1du+R^1zo(sp}E<&g{ zg3boewC$|9Bmv@ff`LQ{=*B%9>da=tO!#yIQ(5Nk;2twIV41n6f%h1-!-=jn*Q5y5 z_R-R@YDl;-xn=`DgVc?3O=ARo3hX4xs|ts0^>D?ao>VGc#SVDhX2c?y;!>hHQESA? z`FXb)GE#)=MP6V3wYGTQeGUA_a#P(WjQ{)PjK6Vq`u7(VxB1Ky{={D%`hU2<3p+Uc z8-@KxahXhADO{5Pkw-u<9bUJ|?H2?0>)VW(L2c~kR(LHGdA*T~Y9H%IEKwRpTKg~KPvfVr?>BF#KZv=+fAiCa5=oG+{$4{;hXii{r%D@*ySgB@%hsPo z)wmW>Hq-Suo|^{+_Uxx?KPTtiq$s%-A=@_dj=;ZiQM8nO38cckbX|o8Ua}jRhD}0` zLsnPi0&CPuxe%{zcm-8;cJ!Zu!O)UAQQPv8=*4Y@j-^mr(LF4a? z^X#8Cu?zY3pl}G8<%?MwhI*XlMiIFk3Y4!wC&tl?AJtHD#djLJn4Al-n_D8Gi!ha@ zprZ%xlCh?sJ?emA=oZrtCW7de(oe|65=-?40?fgQxb`ZCC<<8pQZ(ieslWAco{pc! z8i0KkWh`y!BlX6y%8F=^!s!@5>jEnj`98RQWB^!?6O1PGrxe@ziZ(ruDbim;_z`!< z7!oUuFoaV)Xt_+-P#h**V{k-CK)Nd&n}0`ehDQk)>E+=szdAL)COI>dlIQN>5q3$t zS0S1ybw$n|H5+7|n_r%b>zig3N)O3g6l8~F3?+wxFVG}gB2OS%J`plZ%eI+GjIAlI z>syrVb7<`$zYxJ1R;G`>b=cf7Ap0PrBD%tKN~yCn|`lNbfLE zP=Vw^WI2T;)bd}!2$8LkrD*w(Fa+>{!YY&S+2iz~8LNK&@J{Uf(REhVINSLA#k-`+ z#}q);q!rh=pAp~D<-PHe(e?54$y3KKnv1gXsE(kL-)`DKJkV7dk_4fwUhu)_mhF|h zctzUInyXMdVyB4e3;<()UXi^55Y4qEJDS)a*peJ!U*YIbv{aw#qmaKbz#T3&XGob{ zh|@xV^ycwds?`UKO8dq7A6%rg#aadzodKp@sRwQeiE&aS87kc8lqbrT9nu6CS@hZc zuB>_#!U<2lE?1}~e$@t9N$fMT)qFO#T){7F+f=49UAwlv`v@r^-}u&#(HNUrU^}*J zH%8$|?*Cx50AZXs#j`D1*%PE~xxtV>=`tF2phB>bY-5%bUopyCJ!qPH_ETvt9S?hC zjXrwjj_>)!C8)J3yTbDQXN6GiK{5iC@G@FvypZ#rQR))m4~c5pfI)pfP+7o!KHW8P z&Ba^A+iOMEPfgR66n2!w>-(yTz!~z4`TgrFZ2Bd z+zd8Jt&VD-tnvx$Ds#K*%QxKRSlb6Wok#1S1VE~3aC<0m%Rq^;AZ{}z>98sRga~3E ztRoqbt_8{x#K<@{a>ab4LzpgVlnVnoBmMPmPowjDt9NbJIlZm5^%k8CWzI=_ES%G3 zJF-k(f_5H>nyDg4jw1eMo)acaKTp|joWja&gG;DOg!Sh*^>RUFKc@kzOAJXU5J5rH zucr?7RZho53sw?SG6 zKeI!h`rmoyp`S`@RLeESG@B4t+ETcSLf*EqiH39slzFVUy~22e6gevQu{X!J4t5rL zZNK2gf8JZ#OoBV>%pCU+2DitQOp%=;HzR0nsg{&}n)kt#1_3s z_S6Mu??9E8pw#I~)2&P{J8&L|VTQqGJH(97m}x%3FZc===Vl$tv|_O{_%-zFI5W`D zn6xn+#qi|oIgU4C+sv&#qR${4h~vbOA3D=rKs*zUk9h0Zn_mUxaSjggKU@CZV7ef! z?imo*FESK819(?CJfENp;$8XOu||O-W2>vMhFT2KXE=_lx4}(}yWP?q4uJcz0H^E{ zn{k_{`{X4Op%A2oX9WFfj%kP5XSUX1thf*^IrzoC<~ou4*dx{W4YgFChs{hgPgbk= ziO993c+;Ph@=FxY=-DS{(fW^+^53ll|AmA8Oey|9QbZfozH4DFpnveju(81~TGe2u zyY+(N0}PIdnAiIWTT%)@cMyszC+q(6sq8!ssg>8{9-WM_h_{MCVON@_FG?6tU#t{qRy@p?%jw4&qiueee*$|t~K}EiY6`KM2_x%1G zY=Axr1)@tsBJrxre5J0dvK;A=5I_u3mD{_a{k1madF?(J8$h)Tj@co@SnnyEU3N zu}Hsrq88`W8Ng;g`A-9?F)JOC%T~H{{xuh`NwoSgH>%uCc)!myaC)p07SqL=m)G|a zaAsrA0%}XfXk3ZtV|hohX7S!_xSLX}#NfAamsNv_eY(l2(n+D(lqfYg+$3PE&_3&% z9QKm>R(J-}XsvkZR9o{=ZE&WEuZdD(cvnL)VG24v1lvf`hjc%r;DrmYrt`ll%r##4q)@ zm4_*42qyNbwoa|pZ|cr5jzMJ?V9>APunTE0y*D{Ed##~I7yhaWFlm~h!X^`InKv(S zhFAPvDBCOsO}j0gr-9q<+vPQQ;rD~Ou`E@Fbnf1KCZ#&Y4IWQ&Wgpmngau4CTe@c7 zrB1fL6Q0s1qFy(x*ku_}VKb#Lx)AwrbStXJKatXSFT>1yMBFZot;_|%OajyiiEOTc zUYDHu4E727=OA(%6!G`5lChvXbBwOYV|qs_bmz$?NxlOcI${YyeYCI1GKx6Yv(tS3 z+ofMhJ=jmNN+l2+vUF)sgU`(#aO^I|<;ZNc3NA<*S+g2Ip{HZM=F^;*SQ@T@S(MY8 zpd&qGpg|rDvWQ0dFzPF`$k-cZPN2oJSu)@1@y<001R5-sGdUS111ztkpjGoEd+O17 zwH`Bjlzj5GilsJ9BUWO77v$NhU{e7AG7)lN6UVn|wZ2uK@t%APAHO$<&n+I+$437F zrdQBt%SY5|0I^!MfYUArm69n2mW4Xx zn&*9eVW$5c4Ouassl;thlDcReacnZG=+lHhH9oCGktfkZotO4pxTL2zl1yy)Or#`M z>aaWlc%AD38ZFd7JxfbaUJx&^PC}ljsAWHfRIMjIGMPRbXd0-j?C0l3Z=9h>yqW2) zGd_pVN{@GOx!@j&+U@_i)R@RM{+-M#dAjz7@mt5b&w5zAs?4#-<pYYLIO-(HjDVqp>*DL%aKX8{q8O!Zr-EVOe} zioAmN5);i8u%xAY;j%c_8m7&8f(eUvsvxB3WU00c_ym z{K2=JFHz_Nt!9-2pycTof-iOH)8?KR3MeA7Bv3v(%wv;~>u|#&)HNqB*~~SkentMF zv4k*bNnQ%K%dIogx>*AQj7@_$YNDi=I3B)4XLx+<#K0XIyj1(xuN%R=J5-KXAy-ic}FM+?-KU(o7W)J6O zlB&nu65ck6%oWLQveDLj2B+xP$bU$`sR5`Dh2WbEFe8~)sRZ;$`O1r@qL;z_ns}O5O)V_BgN}ik;pAK}SJ}Oo37=7qsTH_z=pRmBc7PSg zc}R5AJs|wtAp6OVB&lr!lW7W%Z4sr-@kYcbsl@762UNri)7Zuyblt&ZR?J<-90bEJ6SFqtDT;72|Ud3*>kGqUlN+dt&>3-dEE5lu>G^c(i0Q*W@qnj@c; zVo)Zw<3zs*Lys`s#EXT|TQszfKr)cW4D}ddA4OiAbAY+|cFu<_Z2^KKq%GT$IGgT! z-JKslx(`P%$uG2!2XsQ*Ll;{{8XonlKr!(fdti@3h=^>6h$`4`SRmDqHt;kNw4pb$K5myqucgpgpDqYs&y?k&%4OVA6t(%S z8yFH8GJv{rCnqy$rnf7QO~s(_bsiB{G|xrDo+B~l8%CijagYx)@tJ|dvWLV9ae`Z9 zf*Za>3|Qfyh(!3BLa>7Wj2#x~V6r3`f7A}g(Ur7Lh;`gMvP1vxeg>ya_T4#yZmr0J zZTAA}z(X$F1w4O{jJroB}3Jm1U2V>XT{V0bFJTP1nha4aJEFD$bzV%e;KkYrTZJmSU=HvTAM zz8wD2hWzpc0qdWxK9v7=O2ogfK7UWHe{kD>sUH>pUW17t#L)yZJ%1R@W#jvb)z-^{ z%7D-q1){)}+%ppbL;Itrinrdrm(WaVxdFM?D0Trf%HGaQZofW%tJ~m0=k*khq8#U{ znP_))$!L2RJ@N4PKJA{ybRnr_V7ssH+!l$F>U%X42#>7bu9 zWwp?0*r*ip;wT+5apgy;b^Itgu<#8-=O~e>5*Hm?1^9)wya#3Mr%cL1RVLoKvlqj` zV!MBO?7;*HDpWZl;hXo#vU{~-V$~;x6cYo=f(;4#yAa~kl{UHN+6g8NkQ|P=*lx1N zi}yH)pyT+bthkZm>s%I66`2g@uC$AC z&nE%f)7`?-K}ZQg>LJvBu`q#X{W*4L`KWuk^h^qsz)0K)oq|R0DV!rUtppq3NdmbYpM1 zXWakYLdQcu1%?=VZQV9!%KX8cwy!0v&2+-4y%2L36&FqGu5{Vhh{k4r#pWl5l;mQU zcS4?M?@!IQbS_H`4Dror%=6ltluHbf>v*N`33V2JnySNaX2;OUtX^D*7dw?f z1gyWefo&66{dd@~5JL`UEOj5^Ax|WrIcK97sz4Tc2XyO{;^C{TtRskWdKjugh1gsX zoj}+iiE(ya8EYTXy4h@~&q~Gh<;mSpTF$DRo_T$0&sOqzca~z>Z-G4nqoR z5`Guk!Rz5#sJnUWl2jBR2!pn8esMKh+pcu&o|mp%d_QgDdxid@8z?MfjT)6NAL}Nv zOj;(Q3Ug1y)|9?GBgIjHZzjbN5Pzt`M>y5uZFbefh8`g#;Iw3eOAY*BMAqzP&e5w!JrTvAs7*KduWWzfHXSZdk8Hp|jU-n3Q#x zybYd-fmoIrQgr7KkqKW#*bd>Clj}+)zQDaQ6H>%kW$GtWnMM@fNmW6nhAe*t zMEkM=(;n5PpJOzlekschfY1Zh20lV3R^{bJ;0+t>-4*pzMcfLyAoIu45Yi|<06k_B zzit2J^GP8aA9KNY=hbVL3VHX+w`1JUaJqSb{4L9QV&qiU&2lPs+VX$}rkRJJUuVHl zRy-Ip+V5^B=R+P^gSPV~q(BQyzWVjsHN?dcB_M#*E!`9B$u{W()Y`wu3SP+)f`xUH zgs#}TW!z%+`})uqyBzT@nct~;(pLU<>B$&n@>d>#EqK-wlktKSFOBSF({#xBdaR{k z+4}1sj;ZNi`LBQEWq*C)@BUvVuO>Gm z4foDmn;RxGjE#)s_uJU37N3QuHMT_2$x}yg<2xR(4sUlZ#3j@#Mf-NBXAZ;jd4%;U zA|z7ph^_13k_~gqhP%tpgO`X2t*=Ozsy$2`!q@b}la-YUl&!EA4!pjz@wR5qFGNaCDX z{%$&v##&KvKP4HKSVC>@hfO(zqH5yXEWQyOY|J~-6d6*I5|1ukn26`OIj4A&JS-ci zns>UlC;KCGK9EP)tkQm!WxFm6*$`h4b^O#g5a$n5ZUD8EVc7P(+ZnfL2d(;D6 zocCDw4qU^75_fApX~Y})EMSGhN?|p?d zi`q_&?4F%&YhOjEp1jrxy9X0KVJY^OuPb414tub^=ZV;#pyHV`j8*^71>2s+Pu2|(E98e=bC-{l} zWR%6BnvGDz_{9#B?E}4<$&1v1X_xBPef@plB+%Z0Kr#(klJm7sIp=C2#MW;OJR?_O zJ2B0ju16=FLpDxl8)&(0mSOzP9dxWPv#%zgvDpJsfhJD9lp*m^K$eX5?bH|%NiyKA zx`|*iK+LK+f{L8gI$`HpruJmumoF`v6ULxWb!Xm}2F)!Uv;m2D^mt(+_dki^%za&N ze`WD5d}6Hs8IAi7$;|(r%l@~rm9;bHF|Y+#|I1o5O+n+6S&!x}iBOGa%J(!Is7eNo z_qE zAxn^}Z`8fPgv20HB&Zn&h<_BYir4-H5N=Gp_!fp+5kV| zdJpnbQ}L#1Dpw3-x!6b;>$;F44|O=PYvyLYw;B-q%Hk!6^L2PjX<7WIPta<$8K$`i z1*@2U-k4*vArkZoCTgH_bBpO@8&gMd#YuAB%Ed zM+a!kX@)s}#1FJ5Z4Q-^@BWDLcs}(!`C3y(uoXE*BOHs=V)=A+z6>K=7MVw+MXsLY ztqaceV9Pb7@AvpL6ypttP05DCsPaM?{S1_^oGc!x9oA)!W ze8{+X722YGjOHVQ5lb6nRz;1TQNN#i)=C~h9FRr}6Z;BY-LN(CgMW@f8BK@?d!2s} zVFy?{a_WbvEa5+;Vib0)&F`pw1qjecyiC9>-TzFI3FCSV_q`*c&L>a^sDW;_MA$8= z-b)e~89d7GbI9*U_9AYRr&p&@BXv%}vOGS_`2jJ7FiF-TQ=1yyWUhSMst(qTYA21Y zl$8Q&fthheD|H7G7-(yNnsvKL-F=j$ z!EgWSdhmF*8v|QAS%Vx_8lxS?tmx#kYzrp^a`06v`p%DLXc$i`TDki$t90H}qZYK# zxriVOSR60Tkl?O^TcF2K3T~_bm!KBRPskCM&a0@)jSIjmQi zFedbr0KWV6p4-E0=cH&<+tMxT(nJrYuCdC9UNz`6Mi!1OKQ~5{h&5iMzqreT7}I_H z)pH-46m56oNFbF5dTj7ooew^5*exW@;OQprfEi~K?@sM;z5{GV>#iU+*8%lyf=-R) z=8k^HDbzY&iA?L@wtCehJS8oE0>dqMHsMe%f5L5eW706B-fS7?@;$O%sFpMDmE!pd zQ}$i3V>?{1$;*A6pmg5c<~9%ZW&Vc(zX-{a1*8YED1@-)4%xCx|)~2uJk= zh4%;pK?_D_)zUE>_G-YAwIIPz^=YqtYhOJjiELeN{3$;~EyqyBb%+pyZv3KoFfh@i zAC(_anHSJkT~P?qtoT##0%K!fBuKMv$))9L2b#Q6m=C8L1IB2TNCz7mcK zDcsZhh~M6o(O-kSxaZy{pkJH3I>56)J&ZSJnlCD1$FxokEude`z|9_8KEW#VwD(f4 z3G~MoIQy->AMZRb9~})svh#s%m3RuvXRBN0$WUGHmshDR&*$ zzw=vX-%N?GiZ~E63{%eiOky<|V7uW51MRA1FwP(j(-!;4nb%V^BtGk98$i6e5~mkQ z9*2+c?Da<=q0f!u;*p??V)dYp!jd%XpNL2mTb@^;<`A#0!YX$mC<7%;bzGK2(8rM^ z2P2$wnzq*EyNaUxI7`>&nBI^W^e%QMF^4rE9yr|%{LLfnJcu_z;LT1@0#{pc2mSSd zq`zzto#KcTDk6g0Z9L2ULK8}In?h{Nq_taa!V@usY+@3^h1@v8CM_5uW4PDuEi}g3 z@v_R1qbp4kU$Wc3bYD-=HQUA=wkbl$T+FRRI`nw&(rlU3h})o#o)Ga7f0&l_{AbB# zJJXfs_USQ%F@>FGWnE)J`e-hL$TFi{dhVX_0_KGnF}&>WFo@f zs^!Gg`hA63!jWlQh4E^evI+Ut=5-!Ki2=dZ9K7jS=^;D!u?H9@Q2bjOn6)&!=BkMf zCO^A;!7-o|1I&OdrxUJI6LMv$T-_qUK?sApf+W#=Jn#`CtSpL(e&=W!mw*+BP(ZzL3>q5xm3ZS=UEO zZJ7>er$7h)T@jO1UUT={$^Iw5bmjB$*aApBn=*2pAra(TpDm#LmJlzkqfF5RkuDW> zMQNV*cm;jzRu}RdXQWuxm7-Xnox5qDQd?eN38QK2?IuL>8cex1tBZ6cRMTSz*`rhc z5U24K)G}OENTG@iwqW||Ks$IJg9_D?X$BZr)Vnv~sY=}ehM2XWPy^_rJ(iin*az&y z!Ssf43@P@Ar6taFRfYJAD*aD5`Ls0&2wRh?7(mF>Z&O@EV^G%q%4t_1H6WK%RKy>x zC`D$p6x~FFw%vl z>*`vWmxZM;c;m?oDibnpcSnvX1*X7mKolXY^@?&va$q z+S4m%T@ZxZ7XiT*f71?sEbfqjty<|Oj-OvEIvdMOhyt$s#m!B}UWDx`$Q4lYv^X1U zMkFV3#6Od`w!2?LOtKye<1dk4D;7Y;o;MS#VS0(QPPABu)&I5KKD8*Z%syLd7anaI zFl@e*V|k%!RM7G!+T-5#3(ys^Re4v`MwEa~j@TKJ;KF9$!Akj|!IEShAjB0Ka64wII%DlHDF0 zk#f(b;Ot2;({L<9S&hC0=D|pF-!DT9 zj(TcpFI^aPhH`Z0-2c5#W)L6~Up+%Y=^|^9$bDTY2?|bCgkz*piRH#YvWDQ%MN#a@ zHz~4*R>_rL?dqxvd9LagGJb=J;WCS(G%_a;ED1JLvof5Uc)mbW4t*e2GZ;+sWJZyb zEsR{`C(tuLSoYK$Y-?>CGQl6B!zr0&hcvSkUG$`8C_Ty}wowXow`n$qTZ^z{MLpJH z-B__7o1QWaMpv@|)R~SF(z^33wm&|^l z92!6Ljv{MNB3Pqlf-k^qzuvryHV7*vTgxBaUo?zKuNDMukMc2O{WMI}?EJfUMI;Pq zzNM$hwY)AP?VR(3Jb5XT4VF@?c1cqC;jeO3uS#Dt1p==$<(}THK8njtC#D{;AJSv@CzEXG*m#5UkC?T zXUrD{sSnnpjcqoV8{&adZ;rB#u^DHvTc5&KZ%^?Fof)Gs>>8uf0_Bv7PScz~weoK6 zN#-#L%p=kK%CW60%gU@MQ(1BD% z$;SZ0%WRqZPBgn5BQ|_G`@ATx|@Av4omObO)2~!LK<5NFRG13H- z#?H}Cv$D^1S$tXytCX;rmgz#=R%rv?a=cQEaPIuxdUD<^wIA8ellr`lDlv&fAzK~& zr>Ebn(vn+vg0wQBZ5lCItY9_ue>~G#+zOG-3|uEW2l{)ExO={kifK|LXlT$LPpNsp zarZJ{pb@zz+HhIB!*O?GyN_3=-*DK)w*9VGzoX;aYzgw2DMxtIkG>fYP1u!7b@CJH zNPcLOT@Y5;pqTI>efSk?WPDQ87WK-j&W^kHGr+*gG4t(PIQV*r1K+oi`==*28nKjV z%Oe=9IlnhHHeyEzJFw77I-i}1oELP{aQ`rs)J4=>s$ooT7nQd4MVkcX#=_WP;u zdP8k+vn9H;2qYKuTCa&oDDIJvtDm!5H?t$Chf!L+T$&*}kMz%%$lu;8WO23h5X=Yo{O?-0Kw;&BCV=nq4c1YB<*fLPxIB1H3H3eNh14|3vrCgq2V-}FD zuMRkw#O~yB$)|=FTJkdp%d%vGlaGJ83T>+2EuFAB^)@VMUBGgMrG>E+MgCG_rS*Ba zkDpIq<_fdh`l2zSB;~hu(Q_NqUL*CqcWhFjE_BgaATal3JE!}tF!sy57#T+2uMm(a zVnCx1xmDoJKwf{`t9!PQ{-6`Cd%S^pm-8!@d;0ZnX2)a#4<5p~DVoNwB%XSY0_E|O z#`il3tk+Htm+0?R2A?7@t8t0|iG`_x_DePN$@?C&18LM01(j8F!x0PQOBp z)Ly*BQ^z*yoc0NaY8Wp~H2WFQ`fzM%JiC#)E+UOD4~Bq>(mZUslY86SSYR>Tkp^?dwEC7G}Vzf^(w@M)vNQ)X_6v2~dcVSQzI|IoRqL4y0dP3xE4x)?I_iQO! zdg2s!%P5($Yk3H2Jv-mrxU>KlO6w2O@A|%Oe%rX5S0<*cUtUxV&o^Z0wJWGdzPzFe zv+ELWIdHT(fWGjhIZ&H}(aokuSPH2hhQ&Yifs_$CSJ9Q0YE=;UlC>CC7dD4y`6G4* z6;M}O%w@OA-+jQnxksNx@|co9n9JB_(kwSR?JhPh7tk+1j{6rTYU57URR6iWLVmL0 z|MTsQ^?$tHikRD5m;nA(xBd$sjnPAd^lzSG9SXlRLO*$ifZ2nxoRuF#EGJ%Y$f~ahg+OqGMHfQE_NkjrAuPQo|qB}|4 zLbQVJRDLvQs;0X4wxfOMOpfYk{jIo`pafvgJCSC~_~Y#QHV!9QgRN->2?n5eS^jJC z^3w4wlg~vGJx!>vro$L=B@1RxWv=Ep;QbN;fs!z>+twS60m#NfldL-pWJDR>@%4iQ zsqmvDA*BNu=%!$_ES>2kr+=0Gfa0=FK1iK=aYVe_Kh_P&aT(1c71qP4;tKBg^#uPt z7_soF|FU@BMNSap!6g1n;C{UZ_fp;1-OFg5yPN-}3*K7_pqm%#XR24vpiZnaMY+#f z%^00Ej83YALogf%<}}KI;&`8kntJw7JjO?ig%942_ZkXyA<08Gk=og3Li7V2BK&KD zqc2v|=#FSn)AJvzYW-#Nv9Zs`#q1yDtN-(g5^%Hx=-K`shFWO~YBnqUXq+Z3QDwze z+pEjp3YU>8u}RF0#j@!9nI$~8i9Kh2IcDatvX^^PIuSO(a05B|b59WHk5R{zwbzWS-2_-NB=z8W3VXLh)HZLe1(K7zeI+Sv_Fh4+waOI*%J_@Sd9QR)O_HDd382POe^nGd)fBc@2QqlQN6+J zY5tJA`W;_m{8XQe>lJhvgiWii#kc~KccCgtV>$_ghNOf)PvOF}1Z_3}=$|2JdtW?t z*7U&s_CXq^8y_o0RUjbe<*4}8JIADlz5^4aOfYsU|LkY&;*pe!$>5L5<0!&`sp2e) z7CUMw7zwW}6V03>VPs8LlFSEnS?;#uy4Pg&I^>%nIdUp;k8kU|^PNuWw3hHS-_`{b zwXZ&iDz>12kvV274YaYfAjP4F6twpl%Uq~dmZI7_^MuSBtRbp{4A~wV57CBs)JS%*dd?(ua{^B7Xxh098L*P}sEdtz6XGM7TpoHrVf_XuF8Rzf| zTx$fHTDocfd(3g9?-@H?BBn=Qipty$lWNS+e&>bRU;zY|Y?%dS7*m-0=fBgo*GVr5 z?OWa}sdIMrDlWKR`*J><>m6T$&V)o=laBMhArmf9@C@eyznkd34&~AgF~zjx2TI25 zLdP*Ry#V<&4R)+h;>ek$#KUBB%OMf1OE^pDLI2!6I-uK26Y}yN*NIyMqJy+1G=t{W zffoq2532!rv2tI?iF?reqnhGMn%9r~EPBCz#A$f`$M1^&`YPqMGdBH9A^8`ms-R|p zD1`Lh-%y8&=!XJMa#kF73ff zA;XUZk$bIN#9h2S>TDJeBCLNyzhKjT;c7TO`ugyZ;_)RkCzgk%d}tqRNnNQZN!|)6 z23=f7wM!g>MV>TKfi|IE0-4-krZG=jQvlYSLRq?UFtP^w;3|73{;ktheaX71YN(SL z?bF=DhUC(?el!x?SD>eBy8dc*5_qH9ZU)|9vswMo*-sodvI~jt_i}a@jK8Cme%R}N zn#8zeu0BLN!feTE8Ja?hqBYR~<0(^5-7bf!_@i39&};+H$oeidpS=k9`S^UEyw36TzoX7j(^E;9Fp>^x3iZ1D@{F<|RwZ8@b3dcUE;X>ngwS;Tf z;gM(wfW%Uw-N+C161ZiAKAzH=HC_PsxrQ|yQ6-gMvulkqesaQ+8hU}5UqY+eOq1a> z#hfZK6>38Ky|)3Y%aYBR3G%M<21-E_E*uS6|5Qz?2_%>Bd##ru&`SG-@9%rhTNWE> zS{S_4Z{JnuQ%#Rv^-Z#UP6s{D>BQ}#l>OK1Q|FGES&p35&k8ls$6c%N4|OPMMk~r! zsTop9vCDFj@C*d><7x*{kw^~2t7Hk6SqudH$M2BVYXEp25zz((I zuJSGkjLBnyAS9}VQAPdNgZ8%g}psv=yGVXcsmEp<~e-+iU zfE8#Lh$4|7&~D4DX+n}wFVIvaIx3M4si-p0)!ov%gClnT_<O`uD6ZsebUJ1@Y%!W!^z#QiVFaBU>_bu($j<8Q#Zn>`uebEKf0vH0Ze(h$KZ>q0| zf8HETGC<~!I%MN%&Toz2_&X?{mQM$I%G><_TZH7iHcyh>j!<-LtRJOijqwgM(CoHB{c4ZSKx_1Ro3 znc<=h$QH=LI0|fPfcwJU!>UFN(|_+!!Q7NVoKVWSoqcawvN$r6gpAih8IJBVFhf^Y z0;eF&D=TF8%l2-UQ3fZW7V#L^Hr{sH5smgylm^Ijk_O1kl7_E+t7{wojuAcv(FC(E znMLo&vKK|J#LHD`zuUe@wsmI=Nc@lTs8X!uZT%Y=4HE1%UYHr;s~=aU&kWu# zM!KY-r#VSm1kJ*cXzrN+>a-=k$C(VRU8F^y)*YhbI}8-hW(Y}G^GISoTF^;%(Pg19 zXzfT2z%)#hhUd_H=SHqpAY%{z5toVOH4;g3V5oGF^d+Y1LuPByLJVh#a;K*f$+-*e zcS2qQjNH<@Q)mLs&+Os9R_d&RakS^p%5wTge<<>Qt}gk{_{=|4Y)GSRnztZ>W7}%~lE^nuuN| z^vyUuELT*>6;S7xrD(DOuk8Y=%aftdQ!wCXg!TiK{V;~8t1f$1CL0wWcFIbkVG z&<0a&2dcrlIQq#QyZOK-UFj9?Ey!Nu9*x?A!cslU3>r1tF<0kd%0`fNtp-*_&_lEv zCxU}~w@?Cv05qKl^qYD;xN5(cE2rf#tAbFf5qrCZM5x&BjDd=6$aHJmM$7TuYYHUf zCr4M>q+rmT4TlsmlPyhsp|$6%E=Z@Xjd{}Rm87|;@{{_X0+)m{3wQxUE*pD!boXe< zlk&4mOP9XE=*;0wq#J6Xl@m8k`Gm?{KJ56;0xqp|>$_e@B+iT&m@`5Xon_6W`iP}+ z6B-uk2q?((C+@MWsYI1Q>SLRGlW#VhjEzk^n0izJJ<;5X7uG=(B)D&{@#RqWTLHRE z%TY1GBGJ3S;d;vD-iN_@>P%<&Pt)W0!zfD715i;)y5>zV!Yzxd-I@19RC-}8-8nc3 z^K?3DpbKRCgI4kSO=7dG@V8e?n(S^+QXCzeE;VDLTq3bU_*|_8G+f{f{DR3iL`73LKfHJB|n$18G6^(4Hz45A1U%3v0r+I0oD8?(RZ=DI4SJ5miGB{cFb&w!QaCvpn^n zCFnv~7h4qnNr~|-sRk~6w{Vp{gG*6u@5XNJ4yz zTGPy<4TJoMY2ameuTdlf&B<+d?ot5}h*U?HR@1Irb8sZnUktUyQVTG*8Cg1=tf8<3 znv)}PM(Jf&8r^R$q_tFqZf)_eP#Mc|_7HoK^!>TIm_Zw?(t#zqxTn}K!UMPkSBwa8 z(9ObGHgIM52&T%75}Z+``chi5dvd6*;&(BYh4ii#3l1fF)nrc=-N97JB_{`xFSpAI ztDiSFN?_j;h0Akxm}h4P4#nM&^eqzYeVl^m!65d3wQ>ox)P(L(8YcYg@b-BPumZ2p z17W!gJvf8p9Lf%04@ex58{G&0&hQaHD;Q*mo+L9aWaayz)-GN|7oOeg5?*5npH+Tf z6`KkkLy^$_q$i^g4|M3M!ZAq=bVAs_r$Q*249e6YrIHU5v-{JamW7i}4_ z0|E3f*k?D2=iM$3FeGgSs4C_gkWtndeixt$)d{U%?rQ0nSB%wg@9uY?%Q(eOI{Ho1 zvm|EHJs?yoQ#8}?w0cX1I!i=EUo<7IM|Efl4u@CU0RI%YX%3QF0TepRSSqHzBP13C*FzLp=vkU&K`<`s(~s(XE;5FH;Ad7&X!hmf zkDH_7E#+6m&x6Jh+&>+_{~Pk@A5l94>;I-||6h)d|0}YXNfpZeVBSv+rZ3DzMjDRi z>PwaXG6I6IEbKuyup=qmTpopCBKrH~hfDi8>=$%|Z~btH+c`yM)C+mhzEk(PV%%Dk zahB)OCRAso2rN8k<+kOa6n0fH@4u;`fq_`+Hx;n)TW2*@sgN2nVs|=nMFhX#EirE2 zB!ovYyO#S>KfbFEts>@o=zKbT*I$;|8-XBaA)5P|FTT-QT}tOnuVel-yu8e9o|k@( zCcyuQ2mCkAIR8J5KjFU&kK*LbK9l0$-0xLgZd;eVK`7ciRkJLtL1laqLcfYn76&Q3 zw#L>_Hmygw;Jt%xYh8sv%=W_ePke^Ooz~OAo4p|8x>!K=~a zmMFz3eJ@L!TMC{AV)+~M@3W70qg`Z~C@`A-Q z4g6zAFco&aZn_#n3*VpU8nDipsW#qE7sDo<^#Z`w-R&w1Dtj`WLZD}S^z=c-%-UJH zTH5%63-8sCwi1F(+a^djW64!eB@zOnN}WyM43QuQmgUE;CT|wN5*m0~1=m7e{oiB< zKiKCUN>B(B_iN6Wv1s7TYk=pj7JKQ_6NR^ugGy*6a<684A=8hPnWFkI`0S+(jygoZ ztZTFA^v4@UX`N&yp!Z}7kF`lJV8J*@Sdnj#wH`PA$uFZ-fGb(C9nIgp2I+V~S(px6`h8i+(thdkql2EPIk8YkaU_8H|x;a#vAmrSz7>mEUuDs%ZL zzIrV>&>JUaa`Jd}LX^*#lXHfBIx^T9J(qt~6esVQqZc zBCT}UKIJlpsSRiJnN8P@LQS9B+O*~yp=XfCyA#Od&-uHjKjsFo-es`*wIs^CBRv^_ z!w+!!wMKx6?q5dgr{WwsiANbo-zIi%{{gycriJSrKBw}Df3(5*4}Qr1#sSSN^Z}+c z0y=s|f6eET7PiI)#{WK9#>JT_eV!}_K4SP4nO8vgqGge(6pgES6n|A_&#C&wGxw4w zqP%2^(MN7kwxqjNx+8OYRk%p{<)qK(ZQRG)%GI{js6@l*ny#-VHQ8KTjJL5n+*f#L zetgybmO_5r$QFu)lGm8O*(|zq0BK%`)-8dowHY4_(k{Nud`?*93m|Kky{uzYkNUPD zd{u@7AXCgM6t+_8a)3Oi!)n0RGKDy&j~xzgF*^mRzl33_G&{!_8%sWd8el#LvnQf& z?~g3Csn`!Du@2c9mDylFbEC&?rZ&3x#@9sqEAf3+HxAEgrO3wLR%Njj;ndvPVQsZm zIEIX!X{I>2Jn}?1=@PSBFY#m(@=x}m%$JEpqD%l~Qe8jW!)th`LXD;tvmbIJKL$mt z!)QUiPSSmS)w*Sx7QpvX*m%RXp{e7xv_I!~*Lj&f6@UA;_S1GyAAxni!t0bMcIvr? zEf`yZ2C1#pk`D971*{42; za^~Q!hXi07DCWaPA8HDbFcMKC?l5d}$wnj(PT^5mC798f@EmUdSg}_x8mc!1+PaC? zIgf0HQCvvvEQ{jR%us(QON^hJvxKa{;Ob5EkFboE#i%XCAig5&@p})0cN(&MKrS^_ z9sq_>_$xL82Gg-{QwWB8%Mhkx6W*-|Cqc63r)&0B)otomY&Xf^5!q3(sxL7*2Ak)Z z7SrGD9c%&?T>brU($<&SFHAGHKiq$?aaooy+&ys8Mgh>sMZz%3jx+|5g0bIZQPQOv zsO*&ar1AqXg77>BvvRuVh3cS1dv3{JlW;W`0%i;C8wYWIu@wsiR{4xBk4>Jk`v*P9 z*DLpdya8R`5(ME9SP+lVUt3SJLOVNJChxMAF%RNP>Zxvk&4|&!&lG6lzpEWO6WfYK zibsjZoPdu>Zc-X^nKw_de)VNQ_xEE!Cl$15HjzoB!Xkj7Re44XIuJg}YY;x&S%liX z+kkiR$%4Vy$#WxhVO_`Ogz}q?3R3@3ADV74tvW?wm41IX6_bY4=PFvlk7fhgQ;|tu zD%r*xz4grKwQX>RLIFN!XU|9NS-gn1})FIu4bfuPz9|KDcX=0GYYi_5;&Pq>=;L|IVeV0 zmWfctWLc2kx#Go2($>ayJJ#8n`e8@E`R7b8owN1+irlvhwfWc4j|Mo8kX>+Wc<_;D0v`S%87*C)8|fix%!uU5FC@+w zR%TYO%lz1f@BnRJIErF(X$pnvV19!87+T+;(Z*y}vJ6nnNab1=E=a3>eq%4>y4@F! zht&ZNj`fdkl>v2Vc>8+tmw4U0q4ZpW%Au8naPIW;yBVOvm?;H7T{8ChpvOT~&n_3$ zMH+lea)SBMrFqkGd!Y|tT|YW#HN1QXZOoRAqJ%M0ObivsH6~Qgny;gMi#VnZip=-& zDFbPU~>0*YOpzbj%VUWE7JL9=XK^VD%}IA2zvjqg9C(E zQSpQJjb*mca;c(At#YrVUwy9+J>alMJ8c4|+XDdJfx1V^DMD39q2>U`!YCR@)`QO) zOb$3h|HUatMH|j0)Bc#K8Pwct7cy;E?G~~r_4PYA4Qye*u~RhU2%mGhb$C-`6ofd+ zGwUOA(C73s^^eC~ZDA6oN45d!<_>R^ z87TKU+?^2!Cd8t&KLT%P?TQO+Bo*YOCO+PQ7kjOJ@hIi1l=~2}(nSh<%|#Q)Ov=OW zA#BCbR%(cEx*zi=dA{d7z-LvLg13_ZxF- zDYlZW@kXfjl`I9ha)P6ddR5egLSG_!uex~rYVWgZ3zX`?@Mo31VC2u;V+ufVZ zKdO(SvM-5=pAW(1KRyJ~|NIbG+nEAvXkB>-l+0PFu{)=l^~KSsjN#+Dbr^VcZx z*O&f{eEvNm*Hg90LLwo7xR&ILZfvT8f~bjj3TGP=!OP}s#M@H^EU;HQfIVZrZX14p zW?T|ofj#Dg)-1?oMlZ^Dtd5Rz-i)4}j-F3wZMc6S-4sEB&&@%T&w#GRu!bKM_*DDA z+3qqSF>*{^M&hZ{Q}$>8!1JdS{3rc2P@!`GJqR?d`{j}Bt_y@Cj!WfMHN$yX^JaR3 zYYbTaLBW;}r_Oygt9DiO1XvGw1?~gsD)Et+#12?ImDWq~Fm4+;j?FqvysE3Sc?GoJ zMdVnUuS>v@>xU(B#x>N4X^3d(`M;~dLCTAyq>R{4M5}lVUSF?+G8;^X@r#Wxv`I`Vs)DAG8;VLf#->V?W%tZVG9R~ChTb0*g z!3v1R?~-}QYD76g)~v*qY~pL6@9M%;<=tyE$l}JG&Q3*MgaJDQU=3mZeHv1-*0?U^ zEyLv%v(BRw8grF{U;K58JD4dRNpKxA43*Clyppw7?)4>1F^*63sd>gqUiHv;#d~C- zBPvHDKTI0y+2eMLj`94(WA^dzK+}BUHWj;#a1yi0N)^`-?pHqf$fG6N6c{)I()>YN zPal}P$GbH%i_2@MObI$|H12?tQZ)aTd9gQa@E%_7aA>|(Ma1wJO3K|LT|6K7L5>#;$w+04iQEViKoFBZPtY zMo$8c6@$x~J2oM>5j;G(haC;oAmtHKjz`wn9VR#*M%UPD zxINrnzGem%$nZx7=}L1}4;l{&(V_}~-|$V!OiIsQO7{@>TP{^XJH+87j?1I-PMkJt zqm`k>J03AT_^$4`5DOq6xAc#ZC-|#2_~1EgSfcf_K|6Z;3z?#iBF+f2^Hj zbfk;6uG2}!Nyk>lHaqOtwr$(#j$N^B+qP{d9ouH#T5Iin&OYPbv-TK!+~4)9>Z|vg z&(u4AuZ8B-@YudTZ2A_rz@2)}l&P5xFswJ|oK(HS!M>9)!Ch-eb4*Iv=X@vN+j`ZG zbswvKpHht?Shg?FXMID$ZQ;IlqQ3?1U8GbxJ!XB*{{Hy(HPSo2^8G8QMfIOrqRD`Q zGm=H(ASC~Z*DS5Pg>0OBZO%lEW4zKx&FDEXL z&cU07Z+PpG{*4@Sn>#Qo-#x?6PH1K7LGTE{nxi0nR?DMgiMKJJ6$t7ui7)ZhfnxeS z5oou^{s*f_Lq#!91I48Hw_^G?h6n$9nEF>S{cD8UkU`-`{V1f}_#;9Iy_G@a{HPZ8 z0SYPLC*z+H%2kkyGJH*&sSWl8Zqw~t(=L!N6yaAmrnDFMZi+RfoVZmfJG1Ge&mq@V z`Zwm+`=ug!5EI~yF(?K3yHdi)h#;%abMgTUbC7u7;Onq2%##hFFyO}QR5ATVlu`)m zO;o}A^k%C26{|Xl_t>@=N0_zhsn=;7f%bZHSPri6!GxXqP<0wNxUS0F9tS1#HROq; zd#NLMqKG^8Gl{I;Q+Z1N5}_sv?VQk5k*N$*(nJ#48q>{yc49(buyrX#Dcyta1IlGy zq`d`iS~=?X%ODIpn}-mZff1_kH8u6i6suH+8HbI|Igi$34_pbr9V>Ue)+(tj$eI!E zJtkN!FuqYq*Iu^z>x6!r0 zyLkR-c#}5I#C|D`g41e|@45RFw@;x7ZvL}oBRRD} zpPVVX%O?wwl+)ZKYV}1wptt=T=zgyiVJ&*6MX3@#Z!c<$cLz zrW#sY)fHx{nc5&o#yxpGwcqrvQP@v+h-|J8LIcKX=V2Pp8q`4_jrRtwhusWY(q9D$ zs+0nDrN$@si{WcCx~NK$;L?7-K|KR#fdOj5cF^~>LzbSoN(DmSv4=7y$FM z$WbYO=3I-y8!WJ4mSfW>*K}mryVY~fVHS=uK}$D>Yi1Lp-99h_kxiNp>B@K0VK|A!{1h;EPBPB`XadZJ%of1!t?k4e|_LE1{n?VqZ0a6=USTteZoA1fl#r}R%%slzw1 zuFo)xA<`u`9@8>@jw?YHG#ekp*7lr|h*W7B?V zv=Q5Dz4RCxRP3@!K?_<=wZ<_R*r~KcG^scWN$uQKWZ0;t`!-Ja5kk)+_4A1(f-s> z@f-(T%%w3WIcl(wws}rvqukPwrGT*=&ld^%+>{-nb6=CNvolmtoS%u&`0M?&`)nmw z;&5eOffD*6=+w&3wS!b@_Qi2~*!B+zLG3aS4+AA6{I?R~{!bsflx3HEsozD`#;B~BbaW3J;@n-(WG`=!GZ{~)x8`-=?2aFEbbQ{ zooRIrvq?1(FjBMFeyBNlLDSGWthdn~>ne_EE0t5BTDb1myejEOsBf%IS-M*MH6Hpb zV!n4_u=;j})$(bJab3j>xTXjl7MwI(VpjUqOCPUuBU?*&32A;o??cqafC;B`oH=tVv< zbhyR^qSd1hWe?>IF@sx>oI?zeHz$?mm(+j^NYkKXN>ud_iA8`{sh>VD7E+cW8<9vS zPAZ1+A)i^b&2-^YCpU`B5&89_4IZ$bvd6%H>l|THzt9Vw*EzoCm-Cv*(<%Q4SCxuyVkzWA;pc z6BlHO<#`u-ulWb3P70R z!4n`O>nqLPL07H)3PxoUTAWRh`ZFxBz&0*FGBOU!l8OHvBFpO@$LktgtMm8t_XuuIw9t4X`#;Ro$vKK(xE*=ChvWOw%K9BzNb+g=VH9dWZ59laUgxId7n!v0rygF?Wp?xE1)C8bv>*`sVG0F zn&)D*(74$-v_#(aI@LaS6*Q>@Ls~LkddfcPjaM*{uywr~DNn#b@U5X*StHJPyxAmy z&v|_)a(U$~w#5oN(OeY{ZJvFeM<2^w{ zZ{>1uW+Y>}H9Kg+mY<@lq?gEW=}H5*g%VX!9u5yDU#HaBV%RS~<;SqtbY&CFT3-5R8h$8M)6s3ZEx5&Hn^;P-98 zcXG=P&Vz8t_B}?ah6JHiKP*Ajxi=@zH6h85!!!vjbZ*L@rD+!vkO7rWby_BvVh3pr zasxc;1|F@wRO@TytbGqg7UAI)MI%B>e9!3VwQdy6?N>($DFu**cn3GZ=!{H@3{Gj` zznX*M>SB*eTv!G%c>*}`fFpl|eNZC~Me6hua3{yT}x)1uZwSbTX|8Xcdy zUEl;Vk202pDURP_?etF`^O?(z+yZ@f2@pH8u#PXag!l035idrf@u-a0@4GRCuF(Xa z3I&RAE_}?stxEq+#y;=Sksua*tiNmkz9 z&rBUsskf%tuLD_+P5oYZ1jnZLDWh5T@O!qvy6lD+^E!1>!b1V~VNr}iZAF0<$q^i~ z6OljCzKhnjDC75cMEP*yU_2HHg{V!6bFD8)L^D}`yme=Pnw=N}%lHR>YhM5Vh3ft= zi2HA<16Cd65qX;anJkq;6cBVb!a&(9{6TdjHTfc2Hfz%A>ldkQn$ugq!lyo9bx167 zXZkUI#M@s0b0smWQu;^3hw&8VI%eQ3EYKou@9M8iX^a$@DWyP}OR*N8^|iR`$Qsvg@);8kJ!Va z;kWxkFxy%<$qptt6x5BN;$+LjH&+zX?1TIdObEm|JN6(_8fK&4*p>S-r0z7HAtH4$ zebE$Y(6W(z=|=5EJt2fquh?d>S#L$+c`iIu&nE6ekC-d#v zY=j+HiLS|L9ZvMX+Qjp|Tu?4OO;I7I*^yD7!dIONrU+8l3}cr3pU*mf5}nU~6Pbt7$oQ~qMO6E1pNomF#?J1@eiU)0S>l6Mojqd1BuS0{t{$c7D#lRRY0O6 z{1>82+oS~&-4KxIkWm6)iswlFh3N7s8|D9-=t>iIn9hAw{vf)cw4aU9ynhm1#g%wO z>7PVL{9i-|yFL78`@CYkx}pEy5}odU8YcfWDUz|Z`Ab2v4*XV*w*MHa`DcB+L1oPW zM;MVOZ>*ALEYxs_aJv-|QZjM3!9bKGmLWzO7nvk2OtBDD*3eu6zizd_M3vDCOgGvK z#FrgZSSmYA6#t1L+f+3Xl}lMZbaYJ1Lq<01BJ*Mccz@_NkvLa zIxhs_&W<+llrml_ehyR67?;3cp8|JLm&f-uyvWIQ z7wsGY->(()mYEhSuXs$hlI)11`+y2g?A^IDsI5hhnGLS>7-@Vr3kpr+pm~2W|Fgw9 zutWe{{fO+)rd&Ji7w&syC(D+`DFVB-ZJV(#^rtYKzD{SUcmB12m=*NSDNTL3m#6dZ z;!VPdkuGi+f;MJ#aOM}SnJxp4Sd;15{7o?rV^@%@(aYcUBU#63b}2 zrT(L&+#CuYbllmpxyE2m?JsM`<Dlc1@zf|;gA73Ab}u~ck&t8M0~-MIwEZuey z>PYPs#khs-=`x^1#e`tQJr^t+@r`PE*5P#%B6z8l4g$2hbWMs)%dlI)+aYO+0$0I`T}=WE}MV zoRIY+@@T=CIb|7YMKY!b9JN4A6DRc>B!^&Pr6Qe z&Anfrjv>G=FuC8(_g`u7Sny9rr1#bavr^oE{ z!bNmW1*-?I>n<%8E0%PR?{xtvW+fU40+EEBA`5VF$|e0Kn94S{t&&o$C#=In)%Yr5W%+gMTkZu;LErAp_;`F1ifz_oXb1e|6vy6k zOfl}?s)GDQtvGdxaSQ}SdQ4v+>JSsA-w zCh0%ICxRMESy6f}GDB z44n^+7InfHiIpA9lw5$1)EY3>F^66m9fetml*wbl)a>JM0%6bY`Ed|Z=W(?7XZeD& z_$F5t80?OKzJLFFPAKr72D|^c>8kR_-xpX-Z?YakA64$hp|UVfpoM{I{OT$k&MpPt zNqY_>nW$a6v5)LME%ZAQAyzCT_6h7wevo-?7JU@V-2EtJh569yFzt5Y?BTL&>jwxN z5sC=Ppnws+ecTQiUQmQRG>ZObhRuMXh#Yhn%Ao1kPK#22r6pV+`O;h{l`Ob;3YDxPN&um?*<@-0Jwx@Ql7l4cUHgd| z+qYa{kfj-T(magx_<(Og!#GRRwW9iPH-6?8jT9yF3J*dGoQ5l9qpg-FfZQ~INm^;n zGTjvAw3!)rAQ+tm4t1(_&^eci5F&iOb8oWVQOH7lF5lR%w3?gw;KT?U3wkZUgNioe z@LXbzLurh%bp~HjIuvG88T=Nrc;6%Hhu%uK z=!?|g3(h{~%3?+X)p_UOq+mvnd0-<|yvdY5lK+puyxW7NA-aJoUAiJP81K9hlD<<; zF6a7@55>wVQ*!kZrol2PR*k?)J4I+ah7ly)41~)Q*KTewOs0mW>P`v&oqSIh2=Q_c zq~b`)M9af^_l&LP3&mUr4oxhK8R9%VJSL49GGafI3(wwZot49Q>#{blI^@hzwQlC1 z9FGwDy+gQTkA{HPc2`YQj1;})xBxLyar0F zo{UUqBz2`zI*WAb?Kh6#8mHqRMlB2XW%cld;$=&{hN2}-WWRO{-pIK_b)h`2 zDj|-Tolhmvw#2X~{J27(uYCF?qY=!u)=@Zx%u-%ZD!ak^dz+|aa)^{zLOx$4f}mnQ zr)F_H%aCa*kNB)plOlxV$Bk@Wno7vbZk4VYO{r7)$Kz6wPASwUi4LJa>UkEKJfahi zzQZEQ)TtpvrxFW91QR);Vg$VCI4Q)ng7VLP4Y14~_Qno<_;Y3}p+EIe9vE|wf!7}X zcL8($hXVdbyTZ&6n&3t8cWB|Ix2cQkvbM(h1j4xIZRggb z`{5&(?eiYrr4L9pd={12P}B1@p3iz!HW&g$@sMlM%&GslcT~$I^CQ@S4{+>|V<3tE z@){BU%MJ_v_D~)ZzG1|WF*lMBZD#y+d_tA9&7b2-Pmp3Fm!Gs*w`_elAj!g_WsIDHN$CV<} zdX7e8J9)VW09ap-0|73W){>kvjFq zE*kBbQR#Ca5pN%<2rv_?grOLsusU<~Mpxd$*%bwxXyrHXf`N79K?Zrel~H5FP-_sy z?5V{QT;Cwhe`FEn)(a#6O^vqSyZY%%{r zY`OjO53@EKb@4HD@7|;S3XSBpxX1k4CgT#I#d^!^(qV&KqpoOC`JTN98cz+mtLfN+ znn8dsQEgW;pH}>dlfE`Haw|G}v}TroaAobru8V@e)S-QBvE5vzsiEXUM4U6)H?xqU z%f*O|@kkFt$Kv_Ay47o5^5>&4!f}UF`6XJOTf)|DBGrU1i(f5Q=+i8gFPbL2;VrJ4 zfyEB%HBHUq^$fH7P?!3bEteZjW!3e&%BCBW$sDFwDRujfDZT0iaOB#^+bSY(&A@>u z!#B)9Zz{SSJhb;_@d48DQq1n`u(4!C#sGh&%3?Q+TdK{z_J~Jiub&9UUimw~gJEG? zznYvr5|bg(AOI4}njV^^91&5+gQ20F>?t@*Pd_u zg9OT6eta|2F>UmkMrh*e#QMb)MHpGe%e#DKPp}_U`ak5C14VLXV`k%2e#naj4ho_D z=-X1h%T^ueNFNm>;sb3iE5}#aZ|@*5ba``mTeqy(g-$@j!0dx0+2?Vs10u z1mT1n_c|lj3mkc;pmRkRugUh~(Mj3u>3TrL3_EPUD7zW70h%MlVSf_Fc_OgYQWy~O zC0Jk7I_-znO!J&6Y0+LJmnEC4jmi z!6KC*R>zvT-yVYs&^(Z{hfa{%mn=4DJwuL%6}VUNs1yL_u@(6VceSh}OSTx}%UDD9 z^9pVIi=*xUArtY;o&!!rO^d8NKq8lC+f~jPVu4nO+7v0Bf7)=Nr$V(_tM}r&;4hd` z(sX_>M`{wUfYc4N?VvE)B>RbR)87VFqk*aodkDje8tVYcS;h^d>7L99+!zqJlOcV|5ua9ty9v6m zuTrUS$?e$lkm&i)(z6^><_>!+Z+ldJeeR3B8Sxu6mQ?)k;5Rmh9xd)=x63Mev#mnn z&FpJ+FXt1Atky@@O&2Au!vSv#U77H+Hh9o5Fty(h*5V63a1T&JpWImKJxjcstoY0q zZY^1jsNcw1u_S^XprcQ`ma-AE*?ucf=)ZAV5M#5$kkN$a2H{CTyM;RPjIRA98GA?ajmjOOgZrEN6& z*ZtBBe0`8gU`J9COZ7{|2fq8uj4ZR0Q`ScfZHn|3fPy4FnUyw36Z5W!~^D! z)}{f*_HB|TP1#*N7+LQa+}nT5b9zRQ@nTL{N%{&HD0-)Sb9~?eQvUe2x-Fj{A250l z2)Q9&v4pEM6vQltg&5F;q2|09q8AwQwkw0wgX}izrEOCPlf@D-1RG`TC5 z8Uxn%G||n2U}aJ?N$I4WZKwn{8;_ab&;pyzmBO5tG?5Up78Z(@QP$8M7IrGBp>CRg z4VTs^7O4f9dkU9y*}7!y=J>jfDN`2YuI8u6k$3k?-1 zC;JJj1SCWikao@)u|#B(?$0me&6zWphKmwMro24YyjaOm|v!r-a57Uvq?E|z8;!uz!*cy9T1Wnh9U@5IkoLXsbxC#_^u-lw-@iKky5pF3oS;SviV@1` z_z(h7_xSKMC(FHKha$>u1<1TRF_{(F^UIzf`hp>YVrH|RQFj?eFRFn7?W(@RFA7H> zN?>)7YlS>q--&+a(=$4O6`-F@+0GOU;Nyq=_3TR^=Z>3=_w)&#&mODGYpNv3d9E&2 z#C{}-4ZtV>?q!avI{ZL2(t0VFQ`sQ5q*;t}r~Js=j0IW1F(YLk`tY4u*u^I-5B__& z7=q0%z?SWJ7X|ad|8og|5-!Fm+yyv|)-~5+w}c>e&HHe38{*PrI^^`+=+ z?gX5db~3Xy`irv@l(k(^luMgEbjhN;Ac-g${ikuoRrJWkQ?m4w_nBFnyVN3*!e!b3hhc z`T!{y@e_BXIbIUH*;8+K($i=+n%087^pG?z$N0UR*HDy%vP=9 zz-t6a=uFsf*L#a*tclRByMX#yjoh9t%JdH>?gB57p9k%WR#LxrmkP;Hj4`c0u!V!Z zw^1c@A7`F{5{oLn)&6FixN7^=^2U8Vc%EhHz~{uxK5Z}hRnh*W5>JFzTf}pZlAz#9 zio)M+(L1YV0Z+ z-&^MM#tIp7I~QqKPATrtUY5+F3+{TcTa0d;B!Ftagl?gZDl9boX}+IAW%=H(1K_E~dnS>Ii%d>4tFP(1JZ_79f6e#18kyX&q<8m@Qx?luIo zPA^uv#rI{Ge9i^yEzec#?Q1Is&v_veu}ka5w2cnyg_0 zwHRW?j%P1Ipq-IOVqE;B7hisytt71HEr+H%i)a)e-$(CxP~PU$JU$kbog`%!FXi#j zI7tQTjFH%}jUcdpPJS^#TBv_K_yu#f9@$R~f?)}^q`D73=b;U7h&pr#fnf;g!ez9}y~ti0g;ahLp+KQ3n{`H)iiPnC_k~%iL&)Qul<5?b z;YK>$tQXfKN>@6ayZhIvU-nQ5nauCdoSUlQ@<|rPH;)%~via%|;}7xr&wG@g^4COr zu}i+ho^U|x5Y`AIYmjwtk*Jb2I5O(&wh*ZAVfubH^p*IY;u^^D!6RiDln6L#P-F2B znsTrmXq(#nb3!FjFz?V7m_8%@t&o}jUdaFVPV`@TZUE-ZN=s-Tn+(jhZtWCAnEbhF z^1+|d&Ef|ORD`V=f6^l2uncmW;9nUf6V^4eSf9nRnOm&&7cyc>&(e@u$BL|dW|*hd zo-KUsB-u#2cqN#AXnW)d^2KT-cc@ufNJmG$W?Nj78#%)qG@1R+;E=o3~J3 z-cvpUg)V{Ob#t=2z&JN0eim{S7*!_K_iwxvJIvVVyZOmp8UxIrlfWH8pe1>_vb~I% z*JQ9Xub0liEw)Z~Hi95_kiE`8IC$|u2;#F}5NtDrNRpD)Vl@Q0k``sRMtlAa@Uy<8 zo6v(Xs?OXUV>Dgm@9G_d*sQzvNW?J=C@`b0qP^B8M3m%>L6M#LXy+F7$l`GdFze!# zlxov%3}_9pJ(JBdmiT}HPd@?EJkU$B@Y^&r)K?q??BEE!{ty@Nf|^KW&thEy*ff{} zHVyf@ZwqVsgKH50DAO=(Lqg)(p%BfTjT9&HHd+H1w!s~3{D23DB@Ibt2O@i9=s>Zn_%dfR!;YoTpv---As+lJO$5hk>v;RSVhXe=b4 z1s^-90d15TTKSD>fPNl=h@p7d91NB?H99=9cb%6hl)war^Cqg<4lnB0EY6R8t8z zc_DfWRWj6oG_kqKl>UbHWtZ=p(LrZof%mU*eR?YdY;JK|nN zmeU;PC{l`7lRhnpS%MG2K_PpPrkS@iuV66>j_AduoB;v@L3TUIm~EKj!!$?oxBvm; z@?=RS zPC7*9Pb7uUez{8UyMF~^Ms{W!jYD*E3$vCrQR@jyX;H{7aQ_8pL8$>6(SFyYJOIfE zEe4xQi?*2acN}M~R^z5Y1)XzWq20q3`X*!ESPmv0xe^AWGCA!ZRwYlXPqwyqp6?O~ zK!0&mCGWAfq(MVS+0&Z8Khrcy7$#o47gvm2BRfMF1-RUP<@EVUaB{>iG!W(V zhRHXB1Q};D!V>sBZ%m5bmfTG)lKidC8gZ%)DFg8(<`lA7qG2c^C8=8;{1l) zC4a4n*VT7Q)2SGrWdF+%9RsgxsH{8^?KRz_I>*4>pv$k9v10=dQhi?Ojem&Xn!J$T zV~zTY=7`OxYKRO)4R_AY6z7Lu*UkJyJ?9ZeK%W8mN8!#T=ZEi)lcyvxK!c*D82}nJ z6uoY#Iv4A@GI_;(>Dp$!N07BIu6WnmtqZ_z81A>KZv1shF)?Z;;n$giJ1hN1$u*l0 z0u~ZAjXv`Xv)N39kYy>ck*iR6Z#Dh-TSv&(lSPacLejngw<$W zh3uAt)Y`9x=il@uw*#N5ns6Ws!C!f+l2=kuCw%(yAH$+r6t}E}dztJD?+9H?oh#eZ zzhJSC)0x9GN}+hUl$x?y!IEnlB9Lji;uAj5@2(C@{EkQ+_*Q$3-GPp>D^U)d*E6Vb ztKwH#l0^yAj&pQ%yB$2hc4KPoZ&6mqR5-Yh2_k&7%>BF|;n0lQrtmFJVKP$8X-FZ) zF8rxI&86;flvh`0;;mCR3f}H|tOJnsYfUHy+zxH%42wf|s_Fg8?(;Z8#CP+mOoXMc zH$lJf{SI-9ZmTn7rNdbALNa>40MM7ImP4CB9PU(DrqAdA&L0h?HouBv3^7vk$4qzw z$1>89UNrQeHoxB^&O0hdxTOk4eTQ8)6XsoIYBx8xO@#}{0uj1p8!{ok4={O#@;VnB zwk?HLLJ+_a`jm0RdT5^jYp4@uM$in++emR+jS*fg>rq;5ozIi7d*0tW7i6$wp4Uj! zNqVtF z*H5YlhWgA~5~u6#&Y8%vp|6bb%1Lez@*+w&?F|K+{4}j|d2nx!_Zs@u?F5eI)$KJe z+?DcQ3(+6*q7}mfqezJoLG0o33}_0`Z=ju!?AZ-#haC|T2|@WhYLMhd`;#TTjo#RB z^~y{f*8NFT6}|9#<#7i+?LNEOJK$su6?X)cvVb0FLWQJBnVi@7Rkr_CrH`4;*Yi33?KQ8AQ`@;XO?!p!9@ zQX+|uu;HCq{&s$$t~0A$ko8Evr8CB>`6rs(i_g$*0p#AHJcqr5SX$lc!D1msMUBb*~7%?`A#`zoy! zdLWb|B}A#RZT%DrGB)@qE#BY_?-WRRW5w2``itcb`XGfJgLmtuRg}`X_zMj5!H8Cs zSXnV+7DggAS1L&!DSH0RjRen3tg2)4UbJ#!r3#V-iWHbV*eU_1bdh+!t@lveep+Q1 zQMyH2JsN_`=%e8qbZ}45^7bjYC7ZuM&F#oTI0Zppajf|G>GOngUB&0;1eVB?pzK4@ zHrdh4_Kjt$avnE^fdfxZI?P6lVeeeV+0!3dvTe2Vhq`Qpn>~F*4HKZ zDe$|eNYAbMpsuh~B%oG~(b>o{x0-VJB+l?8ZL;8;Gp2+an{OF&OI;>_hjcVM(o0>b zB|WH-^K>$LiZNFr-x^j}_h{6_kAPQY1ODRW0BW1D{(Z&avQ zDz1s3@>q5=3HKr74wWVBEK^j8YJiFRphsmURlIeK_Wn8d~ZHV~Y;w zI_rkDVUrj+4;Hy4sw>>mjM)s~ZW9_I4Fv8C*h+d<`L+tE54jF`BpF!M1_U&~Ob6xK`Z-~;L*p6Tb@ zvm|;#um&X1^Gw@jpA>SFJqPN_(`F_gR&b`*K}kV+Y#08mO~(vX+%F}fa!N0jo3RD% zeN>5B0_zsh2~JpIOc_BR5`e#v}JlYolZ{IwGNL84Ze@xY>cKkGhj&Q`J49Jl)jP_DqznQ zA!Svi4VTV=Qz;}F4(6J)d>KtC7{%FX=f|X*Yl|@>0 zkg!>l(p(?&Kje?YRfb=>!l&o6!0cCsM@d$?iVHGztL*7^^i4}DH91&bh!zHr4GOas z_Xz3a5x#~7$ugE@XX_&*26^K`X;mZ)k@hiTY0cXWkI|B#%C(ih|Hu>h>Z4+W)Al+~ zuqR4uI7xCx=14Rh`>P(H-D)*l+P404wcA;29i|<9ly7s;F zVjQ-l*G7_!>TT}R(2n-}=H+xgjoQTD_KUnY#Ylt(Va>hc?gMO zSMEOiIT2=V11xR*mpn9Ll6xfH?2YeOk*7`&&jaNw`$r$UKOF=2Fc(tZz-dS3p*5mr zTtv!7$I_-5H27Dr6kK*7HkkI00^?^aMj3OZ*s}8>5GY{xRH-{4fB(EuQUGLqZJdN5 z1I*~)1&Mo{3UQO7k&NZ$`1O4M^doY{I$-)-3|J(Tj-!|J`0iPDZX|m~v6K3I2Oquyk)eVQc)Wcphl|f1fJLAL%*^>(yn;97sWm{g1n)*ZP z!nbjrv2|)VZ}y-iEW06)#ro4kgwR%DaFE5Xq1~UAQEwq#B9PBWe1i$Z2AAJT!`8X6 zo=ID4)N%^w=Vvhr558PLoR#s%#~WF^VRkII`+PUygERbI9W4btmmNKnMT`u&@PhH_ zoJpLSca$I9w$zV7sRt7tE3cXUG_C7k1pXosoU%nsZBiyc>j|AeL3CmU=X&|iS6&m& zEwnRmMC1(i@9N3_#yU*c!NJx+!Pw5$;g2^P&`I~d-fIo2nyxsiXdi>MthL-O$gB=K z25qAYt=}vZ>}EMgI4lGN@f!MN6v+3P5(ghXyBmq*X^l1%1U9ZyvL{LA$o1kuVz-c% z<+OKeyn!z6ghHb8`87aoePhNRTjWTB#Am6S`u2Kud*0F2ad>3=`+Zj$#M54TzzG!# zHE%dKzc)mWEX-wGgn|5ELWY+P>ex_#sw_FGSD*kE_Q?&yN@<`^ULJ)LV_=n(P+P^G zG>k)H=i2FW7UHF>NX{=sftUKQDWMwZEaht{)Ttdus?2aA-$P|D-mr5wUO##$O?shg zA{~Y60g9k2QpTrRH!tAhZQ{K}L-rJGSq)hO^blKxxwy%0mutJh)B)YR zo7QAKZeQb_!(^p;qQ0u*kX4&O785#Eqo7vNUisB3V=3Tso}MBsFE}2P0nCoC9NJyw zprdh)ulcQpVel-G&Pw9fkZ0%JXc-8)3|oFG|H8EgnV@3l1WV(-om2+_{xtENu}4i3 zy3N`FVJ@$vAy{kQY|}+Me8y7YH7&dGu!65KQmmu{6T+3^f?t{McG3yx#amFuGcJ^b zd|D+Y4j?X#X!_N`J!308R0bGL$8+gW9i@Ko14cgiav8W2gQs{dmu9DZIZUI{b7lI< z%CDRwvo!7?j=Q{6>ou>9jYf)sd-63r12jx^Q?SQfvO(p2yKPRZ>yuL(bSm48O^y&I zi|jb$TIbNZ6Y3%67njsfi3vLFA!&JSZ~pyu(e%N>8ZrZH{N7|YuZ+(c>eTb2`{Y!$ zvbb$pEZjlo=|4F)5d7swl{51Up>x<>7_Tf+ z+v!32XYyCwquUB=_paj@%I)RbX-|n^ww6y{I?D{?`>3BXgD!U=VJ={y3WJKSl7k3# zl;H__gVC(igQc*20to6J>Y(APw#l)5!j@Ry;Hu)TsWq((3$0Ky)}c=g2Be4bZQ@jY zezIr{Os!xn7FMUGnfB3zM|!N#NVBDeOT~rhnmv*F4CF0tnR1f3^3ozCv-nseK-c17x;Yi<051HkqL4N zc3hcO6-JC(Kib%aqGV>+oQ9H%Qvfv<`b^zAj*!s#8%KDWPE#i)-9+piU!-Y9lggP>?#5M%xy;XtN90{ZZv%Z*ka9$Hmv) zCIjz*mz$|7p2Jvkz0xYnVy|P2FQghp2(wLxVn4;%C!yrtsK^_TbwB@x6<2WGSX{|9mC_W*uya~tOwNnp z+H>8&iM1Uq({ML!Eqh5^V}Ds&VJmeMfnIaNo4Tpp3Q9IxYxg_cJK_hdOjC+-JR)vR zv^?6eL^23?J?7V)+X|+@7?UN=xI{7&MqFVP;JXE1iqJnON;I!>{0#m?iPuj-3v-OP zDS6%(&uV+{74QVL8Eo2LG|u(NSD$y|&Djwl_dp+FWZUU?fn$~x=1fEUR%E6BbRgzw zmQTEiWcvf&zwa904cX_t-Q^)sD(6ut*y3@Tb?oGV6?1wdRm40Nq9eKl^TvV)(s;us zL!jMEpgBEN4ab1!AFvjc2e)ka2FnzRH`^f!ZGnlCuc&_^E>A403AP+4Y8T;xJkKjr zBKY&M&rXB0O6>T3eLE;8$#!WKNyBjoe?J{38fE@Bh^(s7cby$6t}WtY_Gv5+oYk}= z{G#=HbPv?@@yw1Jw?L4>4B5x%ae*9#)R0^@DtF7c_r&9)KOk zNV!Mntc7okdx705V0ATvT%p# za2!)%I7XDavb@OoGHLSK>%2}+G7{J2c&{zV%M4M_gCt_aw+}Xc*+-KuKY5naVpXjl zeP+LcNNTCYc2`}k#b#yTgkyzytBo0ltLM4@1UJc6)6;NJt1nfXBabh8Gc=ZXcs1q~ zUX45QhV>pkJ)6P_A$CCufW$x=X%B@ANLLe5J-{P5Stbiw+t_~KbfFPP?dx5z2p8N`&`e{Z8xleQ6G8k6xwFF zzh14C5$%Gb-P5o}R_T?QfDJ|=#`cv$d-%&z4VNdFprW7h2&aMpS%*M{4%TIz7w<^9 z)J%$_(?Po|Blb-X!Y^VS<1bbCu5_(4D$affk(=>$bc~-dQVb1M8o#V&+t>!=6(_I3lPgj10Q;rG=T!Jx$NB& z`GmuS`XT(vL##^^%jqA$owD7G7~0~?(gm5U*5$`_EU-Ub;|3y*Em2xsmSFMIfrgZ%QVS9}dl62M4iW)YQ626pyAa+@O=lx?XUs6>p zG_i&TOhH~rcaL{&hK5|W!2lk81Dgz|52ow+i@7g`IL|E`t3P(591_RJp%`dJ+ zSY?4O58xTBo{&#}Zfiak`1`W~w>4+~c6avQJdnS-iwd^-M&|!8T}n_5*F^>PBp_5E z5zKo|`9gJ`7&si4G>$tgv05`0Y9-$F4BQx8iR9 zn=3g=;Bj3h52XP^Dty&zAx7TAAq6iTI3CIgXf$29>v#fc29=1Sf*&FWL+>1+iZ@Tf ze2Kg2jJN5)+p9I`Q)O=aMx9y_Rj9jVJ8w&2*Uo&Uj5#~P>MtUcouP9D9R9ZAx7KJ| z^4HpYYzDJYq?jlvbyS^)LniGc`ss=>Dj?t*qBJHXE<*Q4vtm?N>LRlWax3m?e87{l z_sep-@bAHqpUQ;!?P+XM{0n2`L!0o8i_iD$KpT*}>r+kqK7^z3_&k=4_G9MhF<$zy}Y+@x=Jq=kNk3X?dXv%>^;CRkzG(jHGL!myijKqcQ3TPCS zg~L*5)cnBGGV9pFr&LQE3LEMMty_d8H%^%u{6dtgX+QF*fzPmvNHTj)O2VXK3Uas6 zh>6v}bIx)wuhm$ncB5=^f7&_^3qQ~_#WrJWmkoN3Sz<{7)A8UJ#t{>u-Uf6Re&}#a z;pgBoKX-j}ohO8T#n4ZA8>qJE#)>W(#d3N@jHwI|Bt+n>+z$FyRVggy`kE2YMQs;K zrss@#bGiEL59ZDInyglRF7RB{ihXqTWy!Nw-s!5>wL#|Xx!cO-I(^{qkkfhFA~Ua_ zbO`O~naBPQW#1TOX|`>d8D@s94BKXgZQHhOoXD_kM`YNxZQIPS*-=&R_IuTRZ}sRg z&Kcw2*SGfm=C{^fYtCuTI%HjGan#>UIc1Jmj%R)}$b<4@#x}r`%ipVQl7>+anQ9`A zhDcYs2Etw~Ov>H?EVlPbkqPl9hDd~9S8OfhQMGRj0<`Lt?;5io8EEd#5f-L*c$TRv z(xF=oBUZGZFu$TRF`}Kk-tk%6Y9>P-0>0RMg+U%sB0|ctu>^0g#$ySf@hUPj%j}We zJ7+s-<}7hs*pBja2xTuHptHRQwV>8h zQw+YX7rH`9^UZ5Q%vEvm61=T+CkpdD@N*C_4A8mL2I$;76x-!)>o zhL>Q|5RpyP7^>|ik6?DswmJi)R@m*d9N_^7!{SjoIqPb^;fmX~EbHjTj&4Kjo3zMw zP;H@VSn)=A)YS5?tu?DXsmV}I1OM-&F=;=q4yDgZ%Pa6s~z1 zZD=(VONrAi?&qk$EZHfN&giTvTR`ciurYL}s0UYC{JkxeOF8BB_3nygbqw%`X>D#) za3fjNZ~)ASL4(*hg0mFGHN_sLdq4%`p597v*^j9Hy>;YYBvrYi$4QP-24*_@N(#lz zQ`8DTyFeaMLV@DXm@QOZ)T=@u0`B-i<%g3T@ilKmE;Hn;L(u@TWtnBZlW%hFzJWhw z4dCz48@#?P8={K7m>OZ+ecLACtf&H)N=9x?iFrgXf{vIGxFE`{?Ll~lnlWsYN`ea~ z6l_a;kTMY3M6RgNOcCxd~TW%N@t@xokm4$={^#qJPW>=8qxvaEC& zfeCinT}rG*OSP(uWdTBlYK?-Oz+J!1ZK=fP3UX>FkDY)AU6fpzu4gE$clJr0bGWo1 zRp*T-i-f;HUp&vBBbz-{E_CvDB?|f3uOPLJirudlzKv|%8y3$Qv1$kAqKoL_X67EM z`N+o8UArl?XR|$wJhecFun@9?A&$M1ZhKHBV%h;wixOC>HF3*PU2#EV!$Nihl_e7A zZ&$Xp6#%t4!|KdB^cA@yRLYDAFWvQnJ4DI0ov!<$%8Ux%a#71RcalqLWq78KNW%iD zJ;IFq%{o`kG;dn+!p}o4{KXX(r|2voz9Z`fPWIf+oJ-IVw{U>4?FhHvBZ}TBvnp>B zIDx|`oP*y2+ARoUPxGr)&`_`5z?!89Et}@#vH%_Xia-PmK|2jW@Oqm)QMm5Ft|^b#+JHbroHTORAlueGN~JHs!kfvYY(-6 zl(tD$+Fw1y{LYOdTHUAZ*FiXpTCo#trsM@p?_?-?`7>XJz<=x${A>a+BmYyr{Qs>U z|6h792@3yG@T<@g(NKf}5>M!#o2k+r2@rQR`#wDvtN1&PcJ{Yyp_)nD+tCNZS1H%6 zPgzVDmxi>u5SiBDiNVjQkQaJJpO5v8uWUVOP|8(S05p~nS2F`7PE<@(Z8g{NE|VS? zNBB|+VgbOLHoWwjS+3FXgRmkhFB5zLR&CryXF2j?N#6EK5e+27>UFT;~DFr6VWGDC6NTaZ3v z{6_Z}stk8@zAM+mrbT|l_`%Jwj*-PZ1Sw{gsZ?i`e2DJXnknD!J|4jp=#k?0Qtn^B z%e!9-4XAW4_0SArVt>PeW5>6SZO3J{>4OhOPC7qs)rAwBOmrD#u1t@4Gj*$8yT z)f>r6xe>J_7>5p@%Jxtzbk-o7GcV-#HKs^9xAyan6fyLYEPbe>ozb-Vz3E%O!b(>6 zp8ZHktZc@_(8W0?&stldk20w%?Erfxc>r7;5z$y7wvQRKNk+|d#$+1YgGYOODOvRM zF233CkSXa1-)h_Jq`x@)w)Ic`CyabzdCh0K?(#WM^q;vnl>b~q{m}yYpF@EWN>&P3 zO32vQj@KEB@7jBEzG zN8yoKC*eOvu9{mVYN7?jjL+LW&e)o~5IJ3tRQq{Q@JQ`2uBlX)Mm1llpy>5K@iqH(fm4rk{Z+M#*j z>ub{Esm)~g9b>!q(C}ck>!MvFbKm?sv6^|;QISjWsyS37-df$Mm0$rzEXc8R@&?gw zk#c7W6=l72@$s#Ne6_)d4NOAT(m6RN0-)JoiOSp-Ifx0ME7JqXqf!rp zP>kjfLxg2ytK6nE6b+11VZ=%#PdT+&X(1|E^w>5{7834V94dYk4v74X#PfDgTx`1J zH`e5|kCHM{BvG`jbw<(TT6iT(w_c-DIWMSG`8i?r#+z*YlT_iictWUckK#yUt$xOO zr7I}yX#*?0l+`z!)gKEHVo=Y-5&VSB(N<|?mqhi2jZ-xQA?_Npvl4wi zoMLAgLk8!NWPA7Mi*`KRxm+XM884vXG#`jDYXwO^-}f*iZE*y{Pe1=nmHr;Y`Uqa= z$=DVm*2O8t8tM`oNx+rWtKpXh*KVRq_P|IW@RND!2Fm;lyf60$i&j5(f&*^bK`Hnp z5ia9(PNuvDfazGEAM%cLr{GoRNUUw>14@a{?nDU55t&=F2;%|tfT0)NZmf|6}Njl;m7fjBIJ#uZOF1#;i~_k z_gt_=3m)8>3HK;_1-tr_+UB=(xI;|Q1FRzp-j@mqP&EFd-f=M4*%zY=?~dif(zSwo zom^*&p?Xs--7f`sweO`c!~E_9ECe;y_znvCYAwONhoD zNX(%AZVh7R)%S)3mw?A3e*XRHPY<`B0%|mxpRJwAe&kiIz20mQOIq#pL8dScu|e>Xu*gsAa=Q>R4h{7AkuWh}1}y z``04~AKoF`?9=U<8YCR*{vx%mIn;sh@O8LW2f>(d@Bxj`A9wQ8G@PJykkjD@A<17G5^AUMKOY)9Oqd ziym&qbhv|Zxi0G~1xFk}lyZJ>ejw{TSu=qA9A)3KS%8_fWklo;X~u!PZc)lV3Q+6a zTfJMMX@s_H6pxK5E2usBRtszQEwDy|9vf1nHc?CZVxC-fg-**ltt|E4Z%JBxj&N(w z^Q#}WO@z?%_l=oR5cNDL!C?X7UMlMbCL5ON>Go!txA3_pjp|Uu!qj*cS|xHtj>F?2 zapMMV%I%&wG)$#tG2xlSKwJS!M@*IB26Js1$l-5-`r0_zulSJ|(iLvu&0mbE>{vkP zM@^GzTm)NJYe6#7&Or9f0^<$abT1ur(|xN5T)sBuSp6>%`xo@Px?v0jEmFH@THLy3kr|_dRL!c8QT2 zY~r?A%Vi`S)V3vUji5=kyry;LuPB&}BB+d=1_Ck27D zdBJ`@awbYpD>iITz_aDHi&O2SvDPW(Y#N2T3<`(W#O~dBx0Ae$+EZZ*H;yOWAOtl; zBrDOHzEqE*hhdI1(So^7WKYQQ3~u3>NP*sduSmoE$%PkR%D?-haDfjqN|TO1bQ6zF zxiC9l+Ari{0+$WIXVN8pnha|l;l@}F|XSC>6$ zbEkZGF5Jj-8aDDJG>qy*FBWqV#I!R_HaKIVXP(-c*3;uGjy6o__8Bd;XY|5*^8szu zb9l+(?=UGO44yx`iOLCsrTPO?L)EPO;VpQ7ZcD8}e18lY0D`cOekM_Fb4pmD_O<~< zVC?#{SB=h=c4z+yXfOXLf5rL#0JKkC9s>(Udp!dq*-uToKN{A5gSG-?D;Go&WN%~O zZmmRgsXoCth-u{wn2uEvrgCO69U@^+gZeDg*TL+)` zA{XN%BYa+uv*59e6pur<^V8VQ&9`?_Uo6+0ZKee!(?_PzNn0JNtg!uVa!bML5b6lW z&&D01j?iU28&`NX*DAWR?vYX5qLLrog>(`a%5<3Qol$h#FL2fQ2;KZ7O})o0F3xk(Gvn=YNY z;g_(%VwIHLJ&nLb=CW}U>D4_aNERz~@M_}p5UbXlbI0dA#Rbl4@%12!nN;b<4bMiD zyW^A)O&gve>q5acev3Imk@D{EUslSl&HYtuy6yN*;E~HU`F3>+t-4KgE6Ml=&r38I zn^yw}y_&z8Y+iNuumDPvNv%d&5}o;bDc@`u(y`3D==eJ1r{#$Po!U+>8f-Hvd=Y?N9hIkej~*-7%65XdexEaE^(FFS3(!Mxm-*h`4)9teOxi& zNj^&(WvOKNl%(0aYDu1ksPH=ktXM&0A651YF+I$NV4#$Q2#z4RRFPk@%BxB-)A%DwLg&ZPWB(ls|A>jhI%F5Rjd^0<>_{_6S@ncCq%*&)XK! z_@lE7k}(i}MIz5|B-*9u>1j?erIW{;EbAp0{B+&?VjkO_Qvlafyd(E5A^_NWe&ls3 z#=%~{VrI94D@q7D%DDS!EZJ1XJs1EBcBkfG2yPtCR%h+y&vA16= z_duXOZZ(P0q1XhdXdaIlG9=U#@w`|I2OlQl{vH2|voAsZ7HO8gJdXzYS07Nk+kWIb z1H=}%x8w>$eC7m^M?$@@aAIP;bM$SXoN?@LDFmqwwd zp^oI+KuIOZq*m=$6_K^%B`wK+r42Zp1U59Wo-E-@t3z2?0a?x1 zmMF}w1m?hv4UL49hf^C7rI2_c!8wBUfEc4^K9K>uA`OL<_Z^>ey=Y#}>K8heKHeV= zH@_@gW`y3jX7l#)geW7T4%p4A{NlD97| zz=14*^+?)Dw@F6zV8M1wo?YPd;z+Sr0&Hwf3xv*0a*+Nl3mZ zY~s9>Tt%EdpKedwg6WS{9ffX2no~FdHCO$#VKTIOKI< zvRLL?N_Hs-O<>vRpr|lshA!8H%!{5rrWs82^YCJu(37w6O04XH;DF-9L(`m`P&@U zQxKU2$(U_@H=4r2mzRd(E=LeXt z+BXIEfm_HK25-oi!v4`xgh0!m(fv|um`EwGLf5Tu0x^w{ajDE*uPOS_@1qVFI(rbF zPaq@6I~+FA6KQpw#Hz@^zX?%;d_i^W`t1W{@b?tZcNGeMqEIOiO&mh1y=y-~pTH?z zrsx=4tAJKz34-Fu86k*K{uqwA6R6C_Y|{WWW)-_vmt9Trv#i&A0$;hNpJ2D8>67W;EB>+L3a9V@5P303q@pA~bY)f4hWln$>hSbgNc|9BUtCP7bse1VM z^N^?@lxJY@^MJ_yAM36EV6ydpV}+EFgM*&Q-_asNNyA}P5&k_=UtdCQ4SujfA$6fn z(^D+35G^|dGQN-;t;lZ$3BW`57h0HnqQxr-?h#BNb9p5qGI@8+}f z2d`F-GqjJlJ2+pZI%Lf*QU>I;QC%yr)8eqnA}B@6u`Nc=%x*8ePSIUIe(N+dE@Wp3 zPuzklVu6;*R^g;on4p2Kb5pH$o250odaDo53_j~r90`|T7L#HUj(IHb=gF3Z+49H* zN2}SBG8@?~$Mrf%mjJ!$^z_!6(P($NrMx~g%))*=xECwy##`qo=>sfafs5ZK4dnOT zdLzG6zx8U-Zgxk{SQPHtug-X=XZKC3Yk6EsKH4}8PIJB%#P_&P?DooS)U%4I+SGGp zu*U8aD+`G#-!~a78K*UZ!AMkZNnM;zF@q-_a@9Uu`)V?Bw6BDLa78A{6UgSDDkrp%ej^rPw zFjf7S*RTj_W6>^>8?N9RMazPrUOcEMW_%r!64-t>s_+LtK!(#EUYRJH-`ozf+2|Ppbvv$z4D2&SVQWzk4;D$q8 zpKQo}$o86ehYfe0e|-^(Z=WELS6)6P^`B#{LhLi^S|b_mz_`6P`>s*4@E+9S>A!qP zpDrn=>zJr)H`=OV%ZXeJMw-dn(+uGJIw3tJzue2t$FealnV)74K?qJKr{{f_&%2`K zxwjnk7gz-9xj_$nf<@jxf(7BfuVobfl}|YQbsSTm0_m!`sPtZG>}E4C8Ov3JEKyBJ zTzyupJSvthXx}y*ORW)>hu};ioflbP;&wJ2&R{bjA&M_GI|bX4pEm0i=7(;ZPB6?Y zCXg+L^&MIK7}FFwu%1^93c2`c(!&NdLO8TD>LTr;<7I;LaINydmO zU)$Ho(d$({%WmkJ7aZ75tUx$g(?@xDOl_E499qcz(d#El%1Z`s8RyQ*ZMZ$G;RIJ9 zG9R*0mHoTX>+LB2M*~l(#}6Xkmn)~I=c4Q92$V;X4&u>9W>3-~n@tpe3iGQPe8i&z z4!zz>Ce5trP9p+aJ`1VH^HE-;`-}(>Aufgfh{%7wBgFeHJSwq*G9Pc8%ALfrG zk)=drrsAN}l@!I=Zp8|2Mu&0zw3Vq9j#{9OD0A~rS{NUO$^4lS z>O`VHTryGTMx`fh$Pqj?b~R@h)iZA36;plnKCLC-6)YkQ7Y=t9@f&NIk!S4TDj?76 z4fx?!2-!Rm`S_>d%+(&UQ7>`Ly@eZS#2W1VT_kKc3~l%JCaTN;<0=Z*wp1{3TYHI^ zy9zV9Olt4)ah|a|l`|8cxG5bof&_MG!S3}rVtdN29mghHvbc;>=8(Lc30DgJ?67LC z!Dz^BnUB^Yu^MqtFV~Q(9$rLKhHO=HkZOPBn!@_U0fHiVTUvg(Z&-RtD=v7>%^VJ|9b}QE9i5mOF+dIBD6LQk549JW*e3 z;h}qLN`hzY05Ulf+fhZD*1^&kA7)8;ER%N8FezNU=9o;0kuoxAaSd7qky=hE@~MW& z&85z_hKyH&hq`McLl}a#EHQG$I)-=WmR$5^SqhEta3&`9$*MZJwyXM!jS3wd=@}JL zSV^$@)g{`DEOwfD>;8=PE6<1rbt%XVp^xsug`%Fm-JwSEJ3qID2e>^Flqh6wX>wz# z8SsG&O8oW&2`HKKz?N{Ria42Qbr4Q}ll1g*Cc!F~5F$2aT7-=w%IOgIcev7}q=a3cCq#>vk}FVww!wuXy99MrTM}&h zr*}n2iQAP8GhR&KavB$o56iuTH8J9g$ei<}jk1;>7lrQVmqd(@B@g#EjWL%20VPqv zAJZl*0D+$UQZ5-li7QO7GeUMwJfThN-=7Ru%R3-vw1Uj0)_*7zVo9RR3mn(baOcuU z&o+MOIbgHD1wn|t2q!)Zi$0<#iO?``Z1{nb#v)TRLyF@YgR6a(4_j!Krk8vcD41@& z+-11n5Ib*YL6(@pCLMxJt|_dR!l`(xf_95XbhK-N4p8N2*Ur@=d4fn_5bg1=4uJ&n z^jiF+eo~sSppj_==Gvr*Ha(cp#f}t`j=eWA_5b4k?edcuEVFwSjAq0h+YVSQT|by& z@s=Gl{`eMIVG+x`T-~G?OEJ$+r)!mI-mn;JUJ8sS3|!4bVoOlahUJo-%WNypH7#eM z)L6=jZcZ45WsYQCzgU^edW&3>PBG1VQ874&qeR(CSqo~Bb~Xp5UT9svSmVld!W;{% zb>?_Ut=k{lD=nwt*D9=2a7$dyw>}1@7LB{1nsHeda=--I(a}-WFemy9Nh6v5M8yNv6a8?hY~&$Kh}>=nutF+_Df`l& z0}Q}aj3_f|Es5FWImF!`aGO%x1D+DBkh3Z@STH5gUL~Z7&46=HG8Mu3@Vc0t7i%fM zO4np$WA3<-Z9Au?Gh~hCs&$9^NDZ%9#+2H7oluV?)63vcVUxws!plO!O2$E@q+8^H zLA=^@gS1`|)J?hla}LV)Y_rt?xAOa~BT*Y2A_lrP7f(Y=oy{ZHi=j-O791CF!3KbB zb!=`oeOu*ygd#zGofWU?hG=T~iUj-~2 zRZ(7puD1=v_BYtX*G)uK4sSq@Q5($VH)BGx0F@%+(TL z?MuX|xA5~93i=^nk*S{ns!^>N8Pf!LEt+|r`JEV1Q-Q)QSEyX#c^fAKBTx6}E_^8_ z8Q~>GN0z?Wl{1<1H?ML05~s-LErF*1jl=2#h7**Q_PdQyQ+RM5+SXH)%)u3F9glV! z)KkDJ?KXh)>YdUwyyC9i?wi#5ySvZPz-UtmP_PEK1?{de9JY{p`vJvCbVjLY<%5Li zK^4y&R~YQwGZhO1Vx<3#icXHzHd=LaSAB?PWX_%U3q5;SBK&Lt(5WYf&C_6Kx$ayDxH{fcYDxoO!75=%Z`@hJe|N>7)(bN=8p)S9ThwU*Zy~kpYF_2>6N?d&auBP zp*8{@7*Lp9F)n&{>*QyYL_+g^)0;ayZpwIrui)tdvAeXet;%4K2cBTo1#@ss<~pFJ z(cptLF;i@yHyaO{bobPz^HcGFV&fM72%4WP|AC4S^Wd$i1Xrmk(; zcUv3T)ID`L%yygXJ#D5tYUX9nn6jM6TI5~jOM9ad&gfvR(oV%_hAm6PcEqXzlG!4R zU*A&4R}Nkm1>L8Ohn|C-+a>(1_qA67qQ=mgGZ;KSQ*~S;PsP@wPASwJ)n#mFg57a4 z$3&ElJ@~U%Q`VZhzJWHaZ~<$T)gMVJt96mlKQTqqk>2_o;M`rmMKY6Tp=QlIrp`GkL028%_M;Cw5((PQm~K=u%lH7-kOogb z9f<|^ZAtjn3kHu=7?)uGtEYb}rnqM;lbI>uBl49ZUxr7>pvV~&&zUfp%GyNZ;A6RO z%;dYz%I08lO;l~Ym%r}o9&#sImGkEL~SA01kpFKgG3ntVR-iyIsKm9{i#MWHllU%bb8_R68jIx|(?uoS%z zWLo7m1wt>G>NJ5`r@GkhpCpzh*O8iQ$eWkEH64|KX{NS29=6mru&?Dqff{SIpZrTD+)d#$3_^COx+0Dmi@a>IrU6 z@^2Ntlq$iW&OtHRZbaSta2#%;wt7((w#zCrf{)0%&-J&JJW z7lm7Tn1>j~^7Tm#b59xgw}$5D zkFS4_CEgD}E`(vfeCfmfr$Hame_xjUHQb}{$$~VpCt~=kGz(JDvO)axMgoSGSnQY2 zSGV|DJ%4S4Cc6s^8^R}rSy=HZRdbfCNB@rU0ZiMWyAyD=jAztV+iE5&{#?cA zGV!VUmF{uz{(iGd>Pw=HF1p>t3ls&*)tyr_!a)J%sOU83IOoJ=B#O(Kx$AQf zuz-Tb(rY|+6 z6)vw4dAK^WTXDB>Punk~Bg%nSw*vDlE6HOg7NVW#Vep=!LGvJ-Sofplek$s9_0)!f z6sB4%PS2%1>e`eZ9Y28)1s6E4sn=)#E`UC$P9?^7TS8!1r&)ouuYKA@827d&LGvX}5g82B(D0d=ks%D70iZqDKNpgal@dK`$pK z&n!2{gd{USyQ4O8qz`8gU7Xc96#u4^3b5wXf*#6hm{%BO`4KN8s#c+?_Df(hDZl6q zPVJ{$L>?bI63puU{P+37c-kH$Or{<3)M40Xgq>SCy}7_xqiN~gP{BLSB~}Lg&vOl! zF7PI)QVb4VB(>u1LQJ9V*f!=B=z36nr)SikF4sCSLz9rx5YrGTI!5DQ8*ZW0-vbCT zB)$qTk0H{NSzx&UF94m#pyd6ea<;|08l&9VjCkCPK_@2FDu3h8mEHszeyS(H1w^VK zoAo^Szc6u|(!$F_5QOcO-Xr2pSLkIelxG{ym3(KpL;GVdE*FJ@uzpI>DSY-H{`1h; zf8dD-*jU+qdUiS3SPR&HiqYAd=@I?c1Y(4olteEtyqCpnVFaH={elgx(khg2ciIjy z8H5sCg8Rlo!FueudF;IlIhjBBr**J+IxJs-J6&|fa^uZLTZY$%s~51X?>G=cF-Y-V@^<=$M9mCWFX$V$^a~+>`tCz|UR?@U7E5>?vte4v+ z)OJQn)CiA}!lTBRA~iHJ&ir=1^nr6m)m>19D6t3ui%9b?L1yafL@s0#n9RuZNffTg z<7y38Aorhb%@4kb!aW#qv3V|3GIcTo1jZ0HH^9?%)*q{B;e7Ps@Hs-5@wuY^c{S<& zJ+cV7>KQox-%I*0Cf2BQ5AP?$6!fcubh~IC_52jI)HINU>25N0%+dQ15BZH;;jH&k zqk@U%^~PLnSTuIJy}Z2xs14L-s!K+6mHDy5%mUSJ3np|e z+p=_u+$DWp92h^3Lg0W>=h~+Z`LaahAD*T}GmXZk?t4u4A{7wr3&aKUUAOLy*JN6@ z4aMWx6xT&({Pu7FU{MAL#O=J2f#N$G5+lHX*+SNX$~D*~%x1WN>dUx^Ye5gHu;qAj z`nifk_*D8?<<;L==`wtlyEh4jnm4qj0^UNMh=6hdscqWc2+7Tb>6#JY#Kt?51dZJzW@je4gbA zr5agvMAYW?jX;HsRDQB`_GYU-E%*aY1+&wH61bZ7CC1+Hs+x)eDHFv}pLZ6FuYra( z;f~Fy8xk_29+-UcsULc0 zOZlHWX5%0Cl)&Hj)c<~J{p+ItI}Cq*GH|r9ClWL>HWqUIV}$4P+kc&aD~l<7TGYL3 z1Es)&M||C-p^+2F4J4zeD5k;`k_~90jobVBIUqVggz92LqlC+4>NYh)F{w`MH$(CH zgd3^xFh-;kM&>e{dD~drTwJf*B--@xd;-KCBVsf!d@;}KrIYnh>2 z8f-_p%z@Hnc3U%Fl>Nik82ie=w{!AxLajmGGUH~q4KDzSh7281wj6RizB#NAd$tIa zKOk;JfI4O|{7SIJ^km?1nu0_NvzESNC(v4m14R?r^q6`rYesUN0=*%D8dVZX6a&EI zag%B!Ch?tJm2MCzj#z+}(;YxYzTN>WAaBPKu$<0LVy{p2clhr>m^^3v3?UIs0Wx#xejF5@8HC7$GEuW<=x3E+D`=&y|=PS!` zzcoazBteDZ+R|gaoIzc5M!Goi9Bal)qU8eh$hoxz*Soj5MU80%JiQosN!-(}v85>v z;xNu(e_?Kfef>>$G!TbE1yBw$o(SPjVxC30B!VtIg%&o+@A+%n&G5$330?^o?|#-2 zYW1Y&Yv|=vio+g$D)|(M$MNI{ofOtb@N%G5-K-do|NEc$PnV8&LFQ+}3;Z7&UVo=h z@Hsd*S$(Qy*;p%^*;wj18W{??8W{cg!(X@kC!6N4>MFhly8p0w&Cu1mk>8Wj6N6oZpo=AG;1iO#XyIXVe9L8BbaQ>bM|BfzDW6qvf zMvrVC@%3Y5l*Pq;e=@_5&ZuD9z1=wuwK{k3Xcysosrl($S+II!F*MU$ z=!PV19Xa>tvC_Lq|A6cx&qc_p#Lqx<`5k*fpxjZ}&cR*F)lDs%iDUV@*PT|gJMM?H zl`0ZOF@6U*fa5GImx5>2-0TpQE9`{g$0yV1S5F6&8p=0}$oFGzeYVzZwMgX5-rqv7 zL;4{d#LqNU(7@yzNyiibCdBwSq!Bz>q3_Z@T|;Q0233^eyQ+D~BtTU{1Tt09Y9#sF z_$;^JCR+%&aq%N8A!vlSw;1}XNS49ot?rMRxb1imxG@PJ8RVE+n1s$Q_lzezQ?)nv zHVm9WMS%<%IMHFr6Y*h>@z6n5GF3P|{Ad`z2_)p2i{*suVxM}@uPDcxu51M@xE65v z$M^aK7_PJHz7du`mm86shoY2}I8HKD+6;9$w0&o3$7p$Z{8RaxP2i$R^fTq70{y3l zH0s};SN~HGAxP%0x=O)HMdQbboFFnBcZ=(NKDu0$vi|Lk8P-6RUMuBs;Z*0rADa-7 z6zU$)R@WH!hDVpDbK}RCbUhh85j_<>(W4)LmXW6w4iXreCuT)Y_0%Bzv!ePa=YdIQ z>!L{7Hn?d)tk-l~&qdQZ#*V3js86|)dsq%qN3?-?M2=^XXZZB-Bz@WASubZ!X^7z8 ziDQbOB668^HzB(zhQ_0;sB$*xhOBXvJ#>meG!`~_?{6~Flplub_PCHDW(_=7PoV&b`pgSC@@A#zem*|oH>ZgN);de%hK75| zYB$`(Y%2{(vRq&bw!6iTp*98@s2DO-mfhDJi=wk&gc(hgkft{du=Ib1jPwlk_u}vV*U?2Pjh&9f zvN9YnBoiGLB_tecqC@JDgmO?wCf6$zXhP|+0%U7UWvHg+uv#pymSn`1w34?P+LG2E zsP}V^kE0lqSt%C;;+w8h+t!Zo8i!89I$0O?O<4aSPT73z_ zzR^WzD6{(wB^Xhl!K*MMp=S0T(^J^N3j8(I=4BO$&REWxqM~D@fFA(q@Z&0HV4KL$sKWnz;+4)_-H9DHu5LCHJTV{=f`5&~^ta9}hzD)X3%?l)U2> zJ!!-Z1#n4|m-GvGjzgr*C{!?!!qiCL;`&Rs`w1mS?b*_~X1HI@yXllw2mp zBQ4jt*;r8vh{cIQ;zbKDg7Qhh_ZeAA<2z@6?SJC4^gjx^ zUzLKD0+I?kH#DR~q7W!(AC$i=rap6l`ip-(g=mnltiNJ$XqsVA|2W`0%*y1s{tfs| z^#yJPV+G-OF6G_#y;yp(R*ltHaE3mm*~RAkCfy-3$%FOd^<{DM%K&_FQx0FZBNXE! ze?Vk909Wm#iM+QA#zEg+0NNU-8BPsJ$PHB@OEul*#dyLF9VIF62=UUNv-BsTZc(bOq zKU4umbJ6kjjNg#1B73$%5m!cOtql(ukG8D2P>x`PfgYOn2{4n_9NLT%&fzIDp1D0G8;;(7n(ax4GP%L@1t2O@sL@C<&B1gGZE@G83Z59IwbA za=$Kpz8%F%I;y6SG)(j9AYO0TL0T}zPMKp$5_N5vGfM*|o>iAH-$-fq zH|D(@t&av%Ye6LmOf0N>7bslj+#?hu)D_AWWr&4r?I7-@`aC6l`k7_W-d$&n62PzQ zkGryPQXR;(Ty`Tdk5jptNl4(R{jck}1|Xm)LO8_fvHcz#(z!AsomAj-#k;`C9JqQ) zgCdWrd@?wKI2JqldRqMkI7ND;WT&>W)75^JPED7Nom&Cn$Oy+Gm?wr$C}n0K3@9P&J1`Q)x)c~26D_8dj*bD>xBal z6tJ1e6`NDlq>E4LAc?za6$VRoH+>nuja+buK(l_gkYPEm*@=XA`f)PeM(kJin=5~d zyoH^=htkP_J`{I`{L@NIrKvNza*O*mk8bM_&fT8IYTs~?o( zz4P9%Aa**;Bd=I&PY_L45{?(0{~{KSfy7`+?`{?3#HyGOf?1A^S`^JM;w=O@G)S=` zVk~Nyie+yFWu9#dTEJ9cjm5T^zGitku!C^J?&lxk+ zv8CpC;qPFaj6oEiNUgHBUJZW4#Xo19hl6P5e+=dXIu8qZNj`WGi__maBy0)0MWcKJ zy)Ae*hhjCjH6JO5mx9Z3dj4OOyMT7 zQ+2Ca+jUyIAJz|8YpyZoz%zQ6Ku~OzLo4vcgSJH*4u;=Hw>^qP7+xxkyEJ8(LBD1! zov}Y@;r_XO4PLxn;d&f@BF+>mQ5XEE!uZB%3)PEh5*P9$ z0kgLBBu*C5-r3?jP;3bzG$hVOb~d0xGj>R7>o%xE3S{t+>x{HboT}gKpR*Y)w%u

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class b/sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar b/sql/connect/common/src/test/resources/artifact-tests/smallJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..3c4930e8e9549d17b89cd3ecb028f048f331797c GIT binary patch literal 787 zcmWIWW@Zs#;Nak3;B`q1V?Y9&3@i-3t|5-Po_=on|4uP5Ff#;rvvYt{FhP|C;M6Pv zQ~}rQ>*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar new file mode 100644 index 0000000000000000000000000000000000000000..61e1663ad3a2804eb7c39960ea51e56a0ebfaf94 GIT binary patch literal 6119 zcmbVQ2RK|?*B(NQ9wjm&LDVRtw;-aI8H6FaFqj~E??J*Kg6KgIg6L&NZxJwWjyr>u3>s|G!TLjX9nMb}jI{|>(G@K25^a+*?L zB~^KDwQp`fz^4BXN!^!EOla2C^ZYol$O7e0Vnp`UKn<`zM%1|&@KQR={-qY2K zhVcyYPt7}WLV8|vLc!8Nytxdm7%GkiCikkR5Pm0!b257YCHdxne{L=x#G~qID2xIM z(`aZ6#|>$x&LOA|LF9N4w0dUIJ4q$NoaA~c8=yh7V)bG;WrPsJ>1j+@ehtiF{v*!G z!TJaG>kVO88pio}764!;1^~GJCsyM$YJPqm5dkh!Yq*nBrk1|;_{jMQg$te94J5)e zU9xBIhXEpZiC(2V-oqv_mMO-sz)y&@8NFJQDU2PE7PispELnFyhyqb zvA7t=Db9tTpRYwa_7R1tc!}m#*c*Q7d&mmN_fw?Lbxm|whvge8I(&vL*M>lcP4-dD zmJXk7IBxUCN40a)71>Jy#dj6QHKPi-#iFv>bJJUlEMTnMf@X5L=84_mgOo0oooaVQ zR~#iNKzqcFN_ipRIi6dkXV=3AEtCYW-{v(xUzHPF-5)uCGqUJqH~`#c#k+R%4&OCA z-8oYR8OJ#`x>?;A+?en}%=JP6T?+OfG;LhsdXqR!Y56wP1EyQL8 z`LH`vjj!g0p!a|T6jXC%S{gsg$Tw-@;oNSn9^2Bc2j`JB)dd6IrL^GY5(f_ zKu*wD&kpH`VK{rsfn6rH!)xLX89qG&vBrKy;TdJZXZU*qxQA$*8+~i(^eh# z>cxK8<(C~Ggm2Xhcx*6ew^pji7edkk zt3=?1&3+Z!FT3m2Rdta;wc60E_e-LplRfnag`+4jKldG{AyE^=|1~nbA7YM4l4tP_< zxcdepmmy|z@t?*%Ssc-CKJ8Y6b6Q5^8(j3nO-c`YdnvT_VX8I7;UTl@ExA>Y;NiJ8 zoe^|u=4<~Tzga8Y$ZcY5X93qaTsor4SA-9QC!FqfT5zbYQ9P-_E6rwCp>RY#z{9B= zxOXJy;Dl^=zJC?-Xy7);U;P466KLbyk_+~vTxd}X2k!)VY?hL)R^q4q;*E4Jy`bZF z^d47$S2nXLa6pIBL%=RyYtOdKz$;(0!sKNZ-8j4^6y#Vn$9Z?>{A&~F z(RI}O$}^S6(p=Ql!8^k7yoHDYzYJ3a9nJSs=YXTaJV;qv=z6{<*b*Ugtr;KBh}{0x z+a;N|oH*E|>0%}I8A+jv-qcaHiVXz*+s6qMct%%PXURAHU(kfvVyZ@q>Rat(#_|u_ zidT?=ie#0~2PKvca8!>{*L=oG^Xt*roq8%t#s^$rk`$Ba_OX=yy9l+&5Zkt%+QCaol$MRxG-K#kG zC8?Ku4Z918T&IMZW{E^ZbGUWS=a#gn45RGrk0uJ%Nd@U>iRlELtqmn1lwrK9wpSXLrgN+g0Jj* zv3EE~c%muEUzf%XKeMXs0JvFUgST&;kWv63EdPAp2FmMOhxoLL&hU&2T?j*Qx{*al z?@B&1ywS9Xr%3qd`c4|&e7R;T57R8wJB#6FB)@{g&PDjkSG-VrzJTbqwdODD3UkuW z4Q@a4^%uwWMSA#CEs(Bt8H{8})~<8%R4$ai@WmR-kZgi3sgP5}`^63_%~7_RDZ{B% z^`6qY8g7Yz$DelfpIxRuADtYE8nMwKa`Pe@7m}HlxJ{^hJo9-$c~;msDYzVxqL~mc zwwy}5AR)PHE>dl1X+W2o7qqwsJkPqrZ5nU%j_Zc&5v$12bae200y7)gvprxi)$O#c zsg#^f%c%f62do>cpj#A^i^6$I`FAU@5trN8;dyxl0(x)HP9_94Vjtjr|3Cl&luo{3 za>08azwsNAl1-hpi6}2KX66HuCCCQ*tU6uyE3gV)YE_+O6 z0l}3;Bt6XjMOb$E$8*SPqb(N))w*DuTq;GTc@5abE+B+8riLNw-4XmBN2Ab5aJtGu*znWdM> z3Ug{NznfOxB>P5H-;6!>vichPZBx*KGW9gw^Cv6ymT( za}bOgg)W$Cw=eV%tN0KPeD>H1PlwMS0-(T~MNW@f0!7A7L(1P4bZHYL3n>!V&0daW z-xZf>)E6)q5MAZwv^PhO&J-H0w#udFkJQh*%sTD^@pF`FUzivyEEggZMg&*YJH{$B z`@e)RFNZGGY*|N_D7Wnsu&iM*!h$t851 z4X*98(bH?HT1wu^%;uGE>Mg^g;S6GIs?Tq(XXB+xzEc-m!Cj|HwarkIy0x4bx=NYM zC`L%XC+c?BEQV-9j66&}B&KLFi<{BgItcLEkn$6`edX=s=|Y>rsO=-`LP5H8t!M&6 zA7bRRtNnOYVs@y0v}l#Wkaiyq5gw}L&c1QNvQPfQi(dH*R|?0P1#2Kfb~X3#4fd!UQ7~UfH>@iV^EBL|ELj12c_KU@b^$DRN53Q!r1CUpRB-B< z45ZVf-qa%$lDk!C9Kgi7fFW`T91#zL2`pwE6Q%a&r%ucmZ7`ekMtZWa$|AL;ABfFS zKF2hO?itNA9B!q#TrzBHC$42nt@;RJFEEuIZ>SI}<4rp?rR5Jv%BRZfeULTJ*YQcX78*U4^qX2)lR zucz7cDI%B<-@%(X)mrwhQYW@3Br$wilp%CH4g1X5?JK z?VQ8SxGoZbOoq#mnU=nx$EtFVI>v0lsMrFzIBPJf-XZEkzyMdj{$sWWn$r`6`8K5I zuEH4PI$R$PX3w`t%XLgSiBlh(=ja`KJtiAAYi0R1&~Cx-*-KYXoD6Hks0QkqnL$fa zqi_w!1a@|GX+Xiy>OJzsn}Zz$($ZhPct4JVm>jtGPPNvc_C?(&dw`u ztF#q_RUxlZd0=%eQV9pnwQ9MSu+$ou@9=V7(VKb_dgd;ln^iVF9*hyZ>oRtn;Z0W9I=S^rnzlmks#m7JxXgEZ5#{cL|g3O34 z_KiURcDw<3ON)^1nd@Q=uRCB#9I&SvmU=k|M7PibTvhWpdvjnN$D5DQ6Q%7jfX{4&KFPCj#J9#qkw5XXR0GgaplG&a5!YwhS9r#Hraf4&-=7BCb<5Ecs{01g*aAf(s zU|%`UWT7g4ASY?pU5^xAVSx?$XE_|D`OL$rdutr_L#qu<@?QFVRsWBrE{a0bR((U$;K!%5q{E^{7L!R1EopniVEBOO*OK_w~P<8v8Q`w z7uamYS#bwuBY8gRCs&uPrc=h<9U`+_6Ges2-jErMnH}0jy(#;0xa=&roR_p1z6x{Q z)@qv)l#flwa3tI-j<_gz2}W%~BOq6M6~9IAiLMJwnTBQGl7ukOkA#_aA(Enbl(C%y zYrOis-K)ci_>0#24r+4+Cd4uFkmI(yTcw=oQV0FcLZos*{`dejxgAUzii3kC6@j6RZs26hB2lz^&r1Cum%Oqdb4%EWd=~UY<`p53`@oNpLN;yi z%uO)MQ?~2crL%fbuGC*vgtb98EOjZhh`_+kS?)KRyK^7*Y`q$coXd%?d%bb&Xx!#4cD99P%wm3Nkv^TS948;6WTosBVkjyWv`^7 z#lr{V5vbrRE9Dc=uqPo==i%ewn_~Z_i3y%aVi4{QZt$9NNjEoS7|N-r+@tuGgG-UK z;{$|~yHTE*RY5z|*uL87#2qKYuZr8xQ2(mRooa4h?G&+mGCW26P~iR^ zc&fpDwNv1EtUm$2XYapId zKe_&@;QhQazi)1SaxVdYbGd%3_cYF*+}~QApX9%m($l*8)y@Hqj|)it@Zfx}!rxc@ zE13Q}_T*tbJ|6oO{F|!$_iU$i`Kz6>flea(@eF?r+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class b/sql/core/src/test/resources/artifact-tests/HelloWithPackage.class new file mode 100644 index 0000000000000000000000000000000000000000..f0ff0c4f5cf037817f16e0ca49bcc2eee59392ce GIT binary patch literal 635 zcmZuu%SyvQ6g|_{CQV~o<6B!z)J3gmHts4e6vTz7P^q{~(;m-<1MCmPDgs+c4p15Q&Grgh9e7Up0IF7{Vy$2uvEmbLOFxgPe+u0<^TWy literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/IntSumUdf.class b/sql/core/src/test/resources/artifact-tests/IntSumUdf.class new file mode 100644 index 0000000000000000000000000000000000000000..75a41446cfca1f93570ab90a74d80b51e0fb59e4 GIT binary patch literal 1333 zcma)5U2oG!6usBZIJJ`|cG>`GyR;4klD5>eD$o)Z$qA$+N(&)r5uef|aZ{JXCXNGT zrTs}ryDxzDVSm8>sMTJ3LsijA@Gx^{&fI&>o#XL8|Niwi0Bpcb1i9k)jZuHEa}WhY zaB4^VJvVyX=^Bq)M^@V})LgG?@OQj@Xc@zI+?mFBbc@_z>)<@qOBBmw$ zb=~ABVp7l78J2KI&ubEv^gX$vRp$!Lx3^cyJ2R3HV_RnK#hrnD;BTO*L{LYeWU}Ne ztLsHmniMsDap%=WOuQwV=F7DYrEA)tC<@VerTMI%DQK~Ww%2*2Cd5iz*B>Y`R?nxn zxi)AlX}n#=NVnX%Uc|{6F`bz(GIx7vxxT)LkrtQcqN}98#WI84*Bc7^sAaLZ{BW`= zWU-P;i0r9mR#56D;@hjwnOI>d6PugvV` zt60alU<$v>I=;!sv8@zlOjTg@X|g^-hV{s%T(z37A_)FkFs;*pe`pU8Vzm>`$xWn` z&D_s?{`}b_1o3#0=xHYYnem4kIlkSujG}vN4WEvjCT*f{frLj+%RO$oHP`7Pq|Xt} zSL1LAG74OTnK;B@QUL+x=#8JMo61#BcM^~l;&2(}gQ^9D)Ol@6_iV@Z?^CC_{C*s+ zz@h>Su97Duw`0*Wtl5sWJLhFI{!pbfE>*dF^ zV3hz>HHm&|OutLyLAkoL{1s+bXnclWzJmS-VW*|0Pf4jKQYuVCHk4ASnJiU|rKTw* zO#=hhX%22;Pz<_eOJCsHpXc(&|4$h6GguC@L|6&q1l*t)qa2xrGNk1y0aHXOhB~2b L;pIq-K=SQBYqk#E literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/smallClassFile.class b/sql/core/src/test/resources/artifact-tests/smallClassFile.class new file mode 100755 index 0000000000000000000000000000000000000000..e796030e471b058b37fc83f213dc20ad3c034d9c GIT binary patch literal 424 zcmZvYyH3L}6o!8{X>JX)w4_`s!GIJP$ih~{f`k~#MGAs}$!V?1m6M>5fOsqzKw{tl zcqqhi7^+HmIOq8HcW(av`TPQKgnssu+tisbTkm;6u&% zu;JKP^RbQ%hSs@ID&Kj|3pKH^>0=Aq)U8(Uv>543hR(dlRWcLfazTn;9!rtZ$azH% z!QVhKkFIH&l4cS-9WwOl8jCW6{h%R5J%aGRO|Xk5eU?5A5Dtf|OSb`eM!tFQ0rpA( n?M&9HMT3wQe2O)?SEmB*8ya2X9p)VO=m~XIU3ZD%29|#TE;d$_ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/artifact-tests/udf_noA.jar b/sql/core/src/test/resources/artifact-tests/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dw>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/SPARK-21101-1.0.jar b/sql/hive/src/test/resources/SPARK-21101-1.0.jar new file mode 100644 index 0000000000000000000000000000000000000000..768b2334db5c3aa8b1e4186af5fde86120d3231b GIT binary patch literal 7439 zcmb7J1z1#D*XAOFgmkw^#~@uINJ=*%LyqJK2t#+*05TE|HH4DV-Q7rsgi4oO8m=&+ zzz3gKz2EQkKKEbq%sJB?WmMT^==sBelT+RTUu57@i7{b9iv1 zQk!>DU~$cfTY0#TTLmbCb$vDaK>|5f8?#3}GD@37MO()ujkB1P7MD0)K%2~mWI+4q z@{Y2AvvS+A{HISWgnp4F`pUw6j<|P&J)E6QcuaWEzJ>L3^ca_6IXGE=5Bz5j+&?|Q zj$m^e%YSer`d>$9N3fIaKe&_qox3yC?jIo3zk=96-2N#t=6}RldRUsfxk;BBJ-@-KWJ^N#rinTf~4%X6gKbOjmDORleAB(l_w?F9>Q9~XJ@2E!6-9*Gl0>o zg@v=KaIdBFt>}1BJs8`8E}_Q2xeK9bsT^54w)1BiX(aZN9c3kyC&D@yWku->4&O&^ zx9Y)^B^Wwt*H%N72iv2hn@APtT1YzFJ5lr|h@q2U6qqs!tfFKRI|LCs)54NMP?+K^ zu`wbEma1?1ak6-@gkSwBZR>tVemCtCEIn{*|?j8tz4td~>TBI^Z{YnJb7)X8!xb6LfD8)fHC)(G&T}nuonup|g_zYt-?nkFHOP|`O zfvKM~21pUp5n{Q3u)SV+6@q`in;hHO&@4+#oIm@xyLvLfipDA3^hTX5Cdchm_K5hY z?WH8yyLWv*l6WC=%LiwMpWbOWH3mw)SFW)FO?fJUa? zRDl4`LpMl4mmk}`%&^1*T;;c5j2YI} z)X#G#nWwHePY{!4PmxK(i6CAt-Lw`0flCv%;yudCGi^^SMjw&ieURn7*2~|i`hg9@ z2b1piI~_@-C-X3J&@AVT_nuN_zFXMTr@cFy9L`a|ocY4Ajo_Fv|=$ z#qki151<*P>H+{-J83ZTf)-ZP0&5x%VL`R~HEm4pw4ylY0Z}-|;MTUt+htq>vf*ii z(I0XR2NTg%KD)cPNRq<}PO+8dwS@H5Hd9s#cA+Y74k?i2kP;PDLjjWNcpi75Nv62#&m!$q!VL|RtgL`K$BiZI;7HSewdKMiDEJ$;k)r{wfZ#+6s6d{u+OK)Ur7xt^9f^}8S74LE8dI#P=XjW0@j4$ zl4wfUIsu74G+j4Hhc{;Lx@BvVpqFScSl}1JJ=DpFBg3 z7{lL_#?Zwkljpz6c_qv*WI0e}Qj#;#kW6TkzT1|N<0JQ41hf13^VVLKZd_3=Ic3I{ z>rMuqPSuqDA{w3o6*oL#P41eA7fK4${(522x0dwWNtIlJ5sC?v*_s>2<2RE9+vN^v ztWFh>hXpsQ8~VKxsG%#y&=2Dl-f{g=Q`U~j*nS1?(W@V1lOl7DlBI&4bUtf%!%HC` zQa=}bx53BIMqAsLx^Us+iaoUp;E3T#?fwy(l%W28DiwPtmB9?-KH0DjigTmrfHou6 zvZd}^sHO<@%%A(zZlJz*8vPy5aMr*>a!VGwG+NKMM{B=dJ_ ztMX@QxDmJ+7nLu>qJCGQ--H@Z#jT3}J|5;(rIRHLf;O{_EALJ&g6(=&nxtzymsV}a zPT4_JzM~p<(MzPFXyHu8%AJ-MjAQIipu$djgXfB_kO9bY3IQ6H=jh|3Liy zn)yT3V*;y#Q67J9-}8O;;i(Nc-Wxg`+BhCalVoIelU8DP9L>Y`4{2=g za=02VYLcti-qL1_>D7y@c!fuLqL^9thPPx#Olr=1842IHCu4k#1QJ!m?m>eS)Cw2fWfI2RaRPPTyA>(kG(nou=B-+B>8VYe`h<$#uM)lD;;N_~`b~|8ujmd!_gZuLVfkdGH5}IT$ zV>woe4WrA0N3{=wrON?I<#e*k*vHTtQQ+*5U?M1Ht_L2XuJ6EdhCb#bE9j3fCADP~ z-l2F~EW_<3d)n1LHu@a9>rG>PR`6?LO;P3O^J3z1cDPQn_)o`{nM#48gIf4|M|E-U zq9{&?-@n`G_IQ^tSt)154eS_(uj2Dx0V8zbbp6?b8r$Va;##99*F+$ceTeTdgC)!PrGp^_^8=^;_pQDV%mqvI-l zg^t>#M~)>=dn)U63zPKuv&+K@R2qlwe^R49VkJFqfTTH%Jkd*k`q~qkYR%$DzB=J0 z;GHacCRCiLti63iqFp34G6ZLu=eQN#%q-NAyxAu6wp~m78kVywpI@7g{(l)?2V+oaKej1+S+9we<#yOXLqZb~ORj@l zX6V5XZwm|IIPwq8ak7WgQdvW`Fr>8=@mcQH4p}}8-vnZPjaGIM7P;hmRuivoV*nJg zXU-LxtEq%RoaYtm!m?-@iH}#KV-HmahAJx3x}Gg)AWpp->-}9`yA9jK4D$uzvEfql zr#$BwUQhTKBYifd=``cGGuX652$>*ezZ;X42JEn>;~8u=)`7eTh^yl6+#Iqgv2{#U zt+wB_nc7snm8Zcnj#V5AFS< zsy|u_Id5w%>f1@&>KGHd9wt%in``MchaULQ$LQva?UqDN^;xdvjrDa_JZs%2%%d|7 zc~ygT>q!-+q?pZ+_&aO}bZBLYK&m4o&!M#3Ec*lXJLPCP>K&Scs!8tZsa*b=0lb&t z6xs?8@EgHR{Vf>gkExCf@LP!Q)GIr6>DT#nIXiKbI#F~80p}hP5-J5!PgwBD<^q&F zLH=gd>@0z5Oj!hvo#Ql!T333LDJ6C@#$9=v56L`9paN zXy}B{tD4>lJB=eM=22*3e!QG2E&uxj^V5*5mznbLBNyZPyIY>Xc=Of`-Gv#my;n(Z zUgT1W85og106p&HW852*`0`N<>-+OQBYgBTY%a}Q@wbKnIQyJ&>?v6oZ)A;Fbu|xDX3y25n zX|J>5>1DCudaLNaw#)7rugT*?;kOhndW$NtI`)0Mp#~>8*AacS_!$)#a0>C%XeT&8 z+|$poRrMQ)B~m@{bG6o|q#$j2%D{o*o5xvULua}OXD(UHLnULNoB2jG1xD6ro#kgp3_HpoB8QGXM<03M2k2VPu7OY{btVwLN+&$ zFF2BU;|~f%T)McqJW>A3Odo92Z3(RHN2QnqdqJyAJILOwgc*ieeVcCk&w9L6*M$5<} z#PnHn)|WVsqB8j*U`hU3w(ZWe?d_LK)W53>DwJlPkL&~*`v*GxX0PpnhbL&Jw(?eD z3cAQSg*B%t zGcwZ{$F-V-{pMTpO`Q)uP^Y{)PnQ72TyHRpiodO=C8Gq)6c=$qN7bW`t#Ys8CK@?QUPv@LUYV8SM#lY8kF zK75w5*G^C)A2%t}2WR9!Lc@90bK#^BTS8rm%%_#S*B5@>CP`<%POxy*x+W~iK4h1uxFTnlXz4`g;+B)A9&BC&p=~^ zYHCZ=EwUM#f|}Jhd*4$zD?xZgWzfg<0`P60@^f{K;Buexy?SU5cC-9Zh5Ff<{AXu# z2Ya5!7T+sZKVxqF5~HIjt*rv&=i}q$`;Ny#dvIB|^zto}eT$BU=5kr_Fm{;WCH37ajo>sf7umUIbI8w#LHvls9MDxr(fF!yGR7`%Cj_;JVp5M61kbIU z>SlQFejKW2z;2MVi?X?&^C-wvjf6Kzp$X7*^W7$h6xu%XBRIzGO&zhq{^ILhDHvl&D^Mc1 zPL-zqa$g`O@wI`2Tv+wM?yO-JUyn2qK~K!}LC=(wfzbzqpf88;3~`zuy1_Unj)Fs6 zC4q7PO^}(WAq^s>*fDnq>UpwsZY!Y?XAoqDTU!NM5-g|K29>MDz(qwK=h1KzNzfO^ zt^m~%SrG8C!_R8EvCkuSeOL%xdSFqE@=8O#x;g4?n!tU{@*p%8&g3x-S3MnlUFV4l z(K+5YL`TSm7~yjsuDj%q6E-^S_wj?X2z$zdJ;td?7++%|gWb)@tu4;TjRF8ccb#`S z-Kxk$H@&K!J^;=<$DJ)An~JuVmPSRnO96_93;3_t!$S}D?mY|sPJc1Ilivb{jk-W(9_T~hN< zTp`lzjbJm=P(A}3Eh0{+-B8}LPtnlE){4^2E`Zf`3X~snre3odX9v_)K7a+}V6%Ex zq27Y`#RVgcv50O$1YG+G`oS(6A!~cx^AVD4P!Q^Wr`o6^v+2`BT2*UWF&Vu#`Mjp>jiMM9Z74wRyi>*SEKsC&Yy$ zS6`M7zLIo(b>N5ZCL3v+L|j=WMGHc#a}| zor^K;%e8Ma`0JO3S*gU@qsxh5arx){*Av9i$-&Xm$pvERtmCFJqykjtR#MYaEYRIo z<5n43(Z;`i*re=CdP9}$23}D>-=t={n5bNPU)w0Mkh?lxR6uHEN^>NDtC_vEs!gC< zSWHU)HdiQDc|NlMkgpztotSo!a<$mj!p6p|4{%xW{|%k=+OHLh%b+if5N${L>im5L z`T_fU1>?&3%CZC5E|1?~f6+63F#Nn0U5O1}?W*CH%ge9Q!>>cV5+A)Vy!gSB z^^ZJ%l`_6D{=Jy-jWauN?O!tfUuomVK>z*M_m373m!tTXVf)(qSKasDXa5oN_ZGf> zi~nfg=QrspZGJUEw2KSLOL6V{yZ9}MetZ6(!B=VY2iW#!@b{$pYmTe5`hz3mmmL3_ zWWVwKeVYBoxD&YWulW89+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-dlDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB

* A UNIQUE constraint specifies one or more columns as unique columns. Such constraint is satisfied * if and only if no two rows in a table have the same non-null values in the unique columns. + * Unlike PRIMARY KEY, UNIQUE allows nullable columns. + *

+ * NULL values are treated as distinct from each other (NULLS DISTINCT semantics). Two rows + * are considered duplicates only when every column in the unique key has a non-null value and + * every value matches. If any column in the unique key is NULL, the row is always considered + * unique regardless of other values. In other words, multiple rows with NULL in one or more + * unique columns are allowed and do not violate the constraint definition. + *

+ * The {@code NULLS NOT DISTINCT} modifier is not currently supported. *

* Spark doesn't enforce UNIQUE constraints but leverages them for query optimization. Each * constraint is either valid (the existing data is guaranteed to satisfy the constraint), invalid From fdbcc17d4212d81a1c54ddd7a2694c206edcf24c Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Thu, 26 Feb 2026 08:59:18 -0800 Subject: [PATCH 395/400] [SPARK-55692][SQL] Fix `SupportsRuntimeFiltering` and `SupportsRuntimeV2Filtering` documentation ### What changes were proposed in this pull request? This is a follow-up to https://github.com/apache/spark/pull/38924 clarify behaviour of scans with runtime filters. ### Why are the changes needed? Please see discussion at https://github.com/apache/spark/pull/54330#discussion_r2847645387. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This is a documentation change. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54490 from peter-toth/SPARK-55692-fix-supportsruntimefiltering-docs. Authored-by: Peter Toth Signed-off-by: Dongjoon Hyun (cherry picked from commit 366550689fc79d863e8bf8ae60ca994acaffe49c) Signed-off-by: Dongjoon Hyun --- .../sql/connector/read/SupportsRuntimeFiltering.java | 9 +++++---- .../sql/connector/read/SupportsRuntimeV2Filtering.java | 10 +++++----- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeFiltering.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeFiltering.java index 0921a90ac22a7..927d4a53e22fc 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeFiltering.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeFiltering.java @@ -49,10 +49,11 @@ public interface SupportsRuntimeFiltering extends SupportsRuntimeV2Filtering { *

* If the scan also implements {@link SupportsReportPartitioning}, it must preserve * the originally reported partitioning during runtime filtering. While applying runtime filters, - * the scan may detect that some {@link InputPartition}s have no matching data. It can omit - * such partitions entirely only if it does not report a specific partitioning. Otherwise, - * the scan can replace the initially planned {@link InputPartition}s that have no matching - * data with empty {@link InputPartition}s but must preserve the overall number of partitions. + * the scan may detect that some {@link InputPartition}s have no matching data, in which case + * it can either replace the initially planned {@link InputPartition}s that have no matching data + * with empty {@link InputPartition}s, or report only a subset of the original partition values + * (omitting those with no data) via {@link Batch#planInputPartitions()}. The scan must not report + * new partition values that were not present in the original partitioning. *

* Note that Spark will call {@link Scan#toBatch()} again after filtering the scan at runtime. * diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeV2Filtering.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeV2Filtering.java index 7c238bde969b2..f5acdf885bf5c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeV2Filtering.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeV2Filtering.java @@ -53,11 +53,11 @@ public interface SupportsRuntimeV2Filtering extends Scan { *

* If the scan also implements {@link SupportsReportPartitioning}, it must preserve * the originally reported partitioning during runtime filtering. While applying runtime - * predicates, the scan may detect that some {@link InputPartition}s have no matching data. It - * can omit such partitions entirely only if it does not report a specific partitioning. - * Otherwise, the scan can replace the initially planned {@link InputPartition}s that have no - * matching data with empty {@link InputPartition}s but must preserve the overall number of - * partitions. + * predicates, the scan may detect that some {@link InputPartition}s have no matching data, in + * which case it can either replace the initially planned {@link InputPartition}s that have no + * matching data with empty {@link InputPartition}s, or report only a subset of the original + * partition values (omitting those with no data) via {@link Batch#planInputPartitions()}. The + * scan must not report new partition values that were not present in the original partitioning. *

* Note that Spark will call {@link Scan#toBatch()} again after filtering the scan at runtime. * From a7475906a61d265bc5c1657127e704dd0c64ae6d Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 27 Feb 2026 08:55:17 -0800 Subject: [PATCH 396/400] [SPARK-55746][SQL][TESTS] Fix unable to load custom metric object SupportedV1WriteMetric ### What changes were proposed in this pull request? Bug was introduced by SPARK-50315 (https://github.com/apache/spark/pull/48867), won't fail the test, just causes lots of warning logs ``` $ build/sbt "sql/testOnly *V1WriteFallbackSuite" ... 18:06:25.108 WARN org.apache.spark.sql.execution.ui.SQLAppStatusListener: Unable to load custom metric object for class `org.apache.spark.sql.connector.SupportedV1WriteMetric`. Please make sure that the custom metric class is in the classpath and it has 0-arg constructor. org.apache.spark.SparkException: org.apache.spark.sql.connector.SupportedV1WriteMetric did not have a zero-argument constructor or a single-argument constructor that accepts SparkConf. Note: if the class is defined inside of another Scala class, then its constructors may accept an implicit parameter that references the enclosing class; in this case, you must define the class as a top-level class in order to prevent this extra parameter from breaking Spark's ability to find a valid constructor. at org.apache.spark.util.Utils$.$anonfun$loadExtensions$1(Utils.scala:2871) at scala.collection.immutable.List.flatMap(List.scala:283) at scala.collection.immutable.List.flatMap(List.scala:79) at org.apache.spark.util.Utils$.loadExtensions(Utils.scala:2853) at org.apache.spark.sql.execution.ui.SQLAppStatusListener.$anonfun$aggregateMetrics$3(SQLAppStatusListener.scala:220) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.execution.ui.SQLAppStatusListener.$anonfun$aggregateMetrics$2(SQLAppStatusListener.scala:214) ... (repeat many times) ``` ### Why are the changes needed? Fix UT. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Verified locally, no warnings printed after fixing. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54544 from pan3793/SPARK-55746. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit a0a092f9a9938d4e98d6adb5ab501475823f5e57) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/connector/V1WriteFallbackSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index e396232eb70fe..d105b63dd78d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -370,6 +370,10 @@ class InMemoryV1Provider } } +case class SupportedV1WriteMetric(name: String, description: String) extends CustomSumMetric { + def this() = this("dummy", "") +} + class InMemoryTableWithV1Fallback( override val name: String, override val schema: StructType, @@ -425,8 +429,6 @@ class InMemoryTableWithV1Fallback( } override def build(): V1Write = new V1Write { - case class SupportedV1WriteMetric(name: String, description: String) extends CustomSumMetric - override def supportedCustomMetrics(): Array[CustomMetric] = Array(SupportedV1WriteMetric("numOutputRows", "Number of output rows")) From 6073eb7344a3157726f2244accbb72ff14bb6279 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 1 Mar 2026 23:06:47 +0800 Subject: [PATCH 397/400] [SPARK-55747][SQL] Fix NPE when accessing elements from an array that is null ### What changes were proposed in this pull request? The `GetArrayItem` expression incorrectly computed `nullable = false` when indexing into arrays with `containsNull = false` (e.g., from split()), even when the array itself could be null. This caused codegen to skip null checks, leading to NPE on `array.numElements()` during bounds checking. ### Why are the changes needed? To resolve NPE within spark engine. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests in this PR. ### Was this patch authored or co-authored using generative AI tooling? No Closes #54546 from stevomitric/stevomitric/fix-npe-codegen. Lead-authored-by: Wenchen Fan Co-authored-by: Stevo Mitric Signed-off-by: Wenchen Fan (cherry picked from commit c0e367a54b1bb1877d1a367af8d321aca59dff59) Signed-off-by: Wenchen Fan --- .../expressions/complexTypeExtractors.scala | 2 +- .../apache/spark/sql/StringFunctionsSuite.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index dba061eeb870d..f40077c53311b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -431,7 +431,7 @@ trait GetArrayItemUtil { true } } else { - if (failOnError) arrayElementNullable else true + if (failOnError) arrayElementNullable || child.nullable else true } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index ff0ee19ae971a..7bfc8cf4fa611 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -1470,4 +1470,19 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { Seq(Row("abc", "def"))) } } + + test("SPARK-55747: GetArrayItem NPE on null array from split() with ANSI enabled") { + // GetArrayItem.nullable was incorrectly computed as false when the array type has + // containsNull=false (e.g., from StringSplit) but the array itself can be null. + // This caused codegen to skip null checks, leading to NPE when calling + // array.numElements() on a null array during bounds checking. + withTable("t") { + sql("CREATE TABLE t (s STRING) USING parquet") + sql("INSERT INTO t VALUES ('a-b'), (null)") + checkAnswer( + sql("SELECT split(s, '-')[size(split(s, '-')) - 1] FROM t"), + Seq(Row("b"), Row(null)) + ) + } + } } From a54fff253ca3aa8da282f3970839a693a7b2bb89 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 4 Mar 2026 08:21:38 -0800 Subject: [PATCH 398/400] [SPARK-55802][SQL] Fix integer overflow when computing Arrow batch bytes ### What changes were proposed in this pull request? `ArrowWriter.sizeInBytes()` and `SliceBytesArrowOutputProcessorImpl .getBatchBytes()` both accumulated per-column buffer sizes (each an `Int`) into an `Int` accumulator. When the total exceeds 2 GB the sum silently wraps negative, causing the byte-limit checks controlled by `spark.sql.execution.arrow.maxBytesPerBatch` and `spark.sql.execution.arrow.maxBytesPerOutputBatch` to behave incorrectly and potentially allow oversized batches through. Fix by changing both accumulators and return types to `Long`. ### Why are the changes needed? Fix possible overflow when calculating Arrow batch bytes. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.6 Closes #54584 from viirya/fix-arrow-batch-bytes-overflow. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh (cherry picked from commit df195ac59de5bd896cd70699cfe96ebf78bf2976) Signed-off-by: Liang-Chi Hsieh --- .../org/apache/spark/sql/execution/arrow/ArrowWriter.scala | 4 ++-- .../apache/spark/sql/execution/python/PythonArrowOutput.scala | 4 ++-- .../python/streaming/BaseStreamingArrowWriterSuite.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala index 8d68e74dbf874..b5269da035f3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala @@ -124,9 +124,9 @@ class ArrowWriter(val root: VectorSchemaRoot, fields: Array[ArrowFieldWriter]) { count += 1 } - def sizeInBytes(): Int = { + def sizeInBytes(): Long = { var i = 0 - var bytes = 0 + var bytes = 0L while (i < fields.size) { bytes += fields(i).getSizeInBytes() i += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala index df458fa37d7f2..1e8f4ebfd1fee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala @@ -288,8 +288,8 @@ class SliceBytesArrowOutputProcessorImpl( } } - private def getBatchBytes(root: VectorSchemaRoot): Int = { - var batchBytes = 0 + private def getBatchBytes(root: VectorSchemaRoot): Long = { + var batchBytes = 0L root.getFieldVectors.asScala.foreach { vector => batchBytes += vector.getBufferSize } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/BaseStreamingArrowWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/BaseStreamingArrowWriterSuite.scala index fc10a102b4f55..4b44a923fd145 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/BaseStreamingArrowWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/BaseStreamingArrowWriterSuite.scala @@ -76,7 +76,7 @@ class BaseStreamingArrowWriterSuite extends SparkFunSuite with BeforeAndAfterEac () } - when(arrowWriter.sizeInBytes()).thenAnswer { _ => sizeCounter } + when(arrowWriter.sizeInBytes()).thenAnswer { _ => sizeCounter.toLong } // Set arrowMaxBytesPerBatch to 1 transformWithStateInPySparkWriter = new BaseStreamingArrowWriter( From e7aee3ade9158e14997ebf271413e4cf2c328a84 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 9 Mar 2026 14:51:35 -0700 Subject: [PATCH 399/400] [SPARK-55893][EXAMPLES] Fix `SparkDataFramePi` to match with `SparkPi` ### What changes were proposed in this pull request? This PR fixes the `spark.range` start value in `SparkDataFramePi` from `0` to `1` to match `SparkPi`'s `1 until n`. ### Why are the changes needed? We added `SparkDataFramePi` example newly at Apache Spark 4.0.0 to follow `SparkPi` example. - https://github.com/apache/spark/pull/49617 `SparkPi` uses `1 until n` which generates `n - 1` samples and divides by `(n - 1)`. However, `SparkDataFramePi` uses `spark.range(0, n)` which generates `n` samples but still divides by `(n - 1)`, resulting in an inaccurate pi approximation. https://github.com/apache/spark/blob/897e1b828b1a66e0aa7b8a959897fc23f7c29c0c/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala#L34-L39 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This is a simple example fix. Verified by code inspection against `SparkPi.scala`. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Code (claude-opus-4-6) Closes #54696 from dongjoon-hyun/SPARK-55893. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 11069d4cfdeddaa52c092265793a9b05b0a30d58) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/examples/sql/SparkDataFramePi.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkDataFramePi.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkDataFramePi.scala index 0102b2d291e9f..bddd6f9f206c0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkDataFramePi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkDataFramePi.scala @@ -31,7 +31,7 @@ object SparkDataFramePi { import spark.implicits._ val slices = if (args.length > 0) args(0).toInt else 2 val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow - val count = spark.range(0, n, 1, slices) + val count = spark.range(1, n, 1, slices) .select((pow(rand() * 2 - 1, lit(2)) + pow(rand() * 2 - 1, lit(2))).as("v")) .where($"v" <= 1) .count() From 852c60a187da7362689fa9edaec2344dc4d08555 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Puppala Date: Tue, 10 Mar 2026 20:19:19 -0700 Subject: [PATCH 400/400] [SPARK-55848][SQL][4.1] Fix incorrect dedup results with SPJ partial clustering When SPJ partial clustering splits a partition across multiple tasks, post-join dedup operators (dropDuplicates, Window row_number) produce incorrect results because KeyGroupedPartitioning.satisfies0() incorrectly reports satisfaction of ClusteredDistribution via super.satisfies0() short-circuiting the isPartiallyClustered guard. This fix adds an isPartiallyClustered flag to KeyGroupedPartitioning and restructures satisfies0() to check ClusteredDistribution first, returning false when partially clustered. EnsureRequirements then inserts the necessary Exchange. Plain SPJ joins without dedup are unaffected. Closes #54378 --- .../plans/physical/partitioning.scala | 62 +++++---- .../execution/KeyGroupedPartitionedScan.scala | 3 +- .../exchange/EnsureRequirements.scala | 6 +- .../exchange/ShuffleExchangeExec.scala | 4 +- .../DistributionAndOrderingSuiteBase.scala | 5 +- .../KeyGroupedPartitioningSuite.scala | 124 +++++++++++++++++- .../exchange/EnsureRequirementsSuite.scala | 2 +- 7 files changed, 166 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 101d13c6b580c..fd71e22c555cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -379,36 +379,36 @@ case class KeyGroupedPartitioning( expressions: Seq[Expression], numPartitions: Int, partitionValues: Seq[InternalRow] = Seq.empty, - originalPartitionValues: Seq[InternalRow] = Seq.empty) extends HashPartitioningLike { + originalPartitionValues: Seq[InternalRow] = Seq.empty, + isPartiallyClustered: Boolean = false) extends HashPartitioningLike { + // See SPARK-55848. We must check ClusteredDistribution BEFORE delegating to + // super.satisfies0(), because HashPartitioningLike.satisfies0() also matches + // ClusteredDistribution and returns true, which would short-circuit the + // isPartiallyClustered guard. override def satisfies0(required: Distribution): Boolean = { - super.satisfies0(required) || { - required match { - case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) => - if (requireAllClusterKeys) { - // Checks whether this partitioning is partitioned on exactly same clustering keys of - // `ClusteredDistribution`. - c.areAllClusterKeysMatched(expressions) + required match { + case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) => + if (isPartiallyClustered) { + false + } else if (requireAllClusterKeys) { + c.areAllClusterKeysMatched(expressions) + } else { + val attributes = expressions.flatMap(_.collectLeaves()) + + if (SQLConf.get.v2BucketingAllowJoinKeysSubsetOfPartitionKeys) { + requiredClustering.exists(x => attributes.exists(_.semanticEquals(x))) && + expressions.forall(_.collectLeaves().size == 1) } else { - // We'll need to find leaf attributes from the partition expressions first. - val attributes = expressions.flatMap(_.collectLeaves()) - - if (SQLConf.get.v2BucketingAllowJoinKeysSubsetOfPartitionKeys) { - // check that join keys (required clustering keys) - // overlap with partition keys (KeyGroupedPartitioning attributes) - requiredClustering.exists(x => attributes.exists(_.semanticEquals(x))) && - expressions.forall(_.collectLeaves().size == 1) - } else { - attributes.forall(x => requiredClustering.exists(_.semanticEquals(x))) - } + attributes.forall(x => requiredClustering.exists(_.semanticEquals(x))) } + } - case o @ OrderedDistribution(_) if SQLConf.get.v2BucketingAllowSorting => - o.areAllClusterKeysMatched(expressions) + case o @ OrderedDistribution(_) if SQLConf.get.v2BucketingAllowSorting => + o.areAllClusterKeysMatched(expressions) - case _ => - false - } + case _ => + super.satisfies0(required) } } @@ -420,7 +420,7 @@ case class KeyGroupedPartitioning( // the returned shuffle spec. val joinKeyPositions = result.keyPositions.map(_.nonEmpty).zipWithIndex.filter(_._1).map(_._2) val projectedPartitioning = KeyGroupedPartitioning(expressions, joinKeyPositions, - partitionValues, originalPartitionValues) + partitionValues, originalPartitionValues, isPartiallyClustered) result.copy(partitioning = projectedPartitioning, joinKeyPositions = Some(joinKeyPositions)) } else { result @@ -435,7 +435,7 @@ case class KeyGroupedPartitioning( } override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = - copy(expressions = newChildren) + copy(expressions = newChildren, isPartiallyClustered = isPartiallyClustered) } object KeyGroupedPartitioning { @@ -443,7 +443,8 @@ object KeyGroupedPartitioning { expressions: Seq[Expression], projectionPositions: Seq[Int], partitionValues: Seq[InternalRow], - originalPartitionValues: Seq[InternalRow]): KeyGroupedPartitioning = { + originalPartitionValues: Seq[InternalRow], + isPartiallyClustered: Boolean): KeyGroupedPartitioning = { val projectedExpressions = projectionPositions.map(expressions(_)) val projectedPartitionValues = partitionValues.map(project(expressions, projectionPositions, _)) val projectedOriginalPartitionValues = @@ -455,7 +456,7 @@ object KeyGroupedPartitioning { .map(_.row) KeyGroupedPartitioning(projectedExpressions, finalPartitionValues.length, - finalPartitionValues, projectedOriginalPartitionValues) + finalPartitionValues, projectedOriginalPartitionValues, isPartiallyClustered) } def project( @@ -867,7 +868,10 @@ case class KeyGroupedShuffleSpec( // transform functions. // 4. the partition values from both sides are following the same order. case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution, _) => - distribution.clustering.length == otherDistribution.clustering.length && + // SPARK-55848: partially-clustered partitioning is not compatible for SPJ + !partitioning.isPartiallyClustered && + !otherPartitioning.isPartiallyClustered && + distribution.clustering.length == otherDistribution.clustering.length && numPartitions == other.numPartitions && areKeysCompatible(otherSpec) && partitioning.partitionValues.zip(otherPartitioning.partitionValues).forall { case (left, right) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala index e9d8e8e6d0fb3..0a70021dc858c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala @@ -58,7 +58,8 @@ trait KeyGroupedPartitionedScan[T] { } } basePartitioning.copy(expressions = projectedExpressions, numPartitions = newPartValues.length, - partitionValues = newPartValues) + partitionValues = newPartValues, + isPartiallyClustered = spjParams.applyPartialClustering) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 088ece6554c96..0d180bd336221 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -307,7 +307,7 @@ case class EnsureRequirements( private def ensureOrdering(plan: SparkPlan, distribution: Distribution) = { (plan.outputPartitioning, distribution) match { - case (p @ KeyGroupedPartitioning(expressions, _, partitionValues, _), + case (p @ KeyGroupedPartitioning(expressions, _, partitionValues, _, _), d @ OrderedDistribution(ordering)) if p.satisfies(d) => val attrs = expressions.flatMap(_.collectLeaves()).map(_.asInstanceOf[Attribute]) val partitionOrdering: Ordering[InternalRow] = { @@ -340,12 +340,12 @@ case class EnsureRequirements( reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys) .orElse(reorderJoinKeysRecursively( leftKeys, rightKeys, leftPartitioning, None)) - case (Some(KeyGroupedPartitioning(clustering, _, _, _)), _) => + case (Some(KeyGroupedPartitioning(clustering, _, _, _, _)), _) => val leafExprs = clustering.flatMap(_.collectLeaves()) reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leafExprs, leftKeys) .orElse(reorderJoinKeysRecursively( leftKeys, rightKeys, None, rightPartitioning)) - case (_, Some(KeyGroupedPartitioning(clustering, _, _, _))) => + case (_, Some(KeyGroupedPartitioning(clustering, _, _, _, _))) => val leafExprs = clustering.flatMap(_.collectLeaves()) reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leafExprs, rightKeys) .orElse(reorderJoinKeysRecursively( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index f052bd9068805..c59bb4d39b096 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -370,7 +370,7 @@ object ShuffleExchangeExec { ascending = true, samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) case SinglePartition => new ConstantPartitioner - case k @ KeyGroupedPartitioning(expressions, n, _, _) => + case k @ KeyGroupedPartitioning(expressions, n, _, _, _) => val valueMap = k.uniquePartitionValues.zipWithIndex.map { case (partition, index) => (partition.toSeq(expressions.map(_.dataType)), index) }.toMap @@ -401,7 +401,7 @@ object ShuffleExchangeExec { val projection = UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) row => projection(row) case SinglePartition => identity - case KeyGroupedPartitioning(expressions, _, _, _) => + case KeyGroupedPartitioning(expressions, _, _, _, _) => row => bindReferences(expressions, outputAttributes).map(_.eval(row)) case s: ShufflePartitionIdPassThrough => // For ShufflePartitionIdPassThrough, the expression directly evaluates to the partition ID diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DistributionAndOrderingSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DistributionAndOrderingSuiteBase.scala index 1a0efa7c4aafb..80f360ed216f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DistributionAndOrderingSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DistributionAndOrderingSuiteBase.scala @@ -51,9 +51,10 @@ abstract class DistributionAndOrderingSuiteBase plan: QueryPlan[T]): Partitioning = partitioning match { case HashPartitioning(exprs, numPartitions) => HashPartitioning(exprs.map(resolveAttrs(_, plan)), numPartitions) - case KeyGroupedPartitioning(clustering, numPartitions, partValues, originalPartValues) => + case KeyGroupedPartitioning(clustering, numPartitions, partValues, + originalPartValues, isPartiallyClustered) => KeyGroupedPartitioning(clustering.map(resolveAttrs(_, plan)), numPartitions, partValues, - originalPartValues) + originalPartValues, isPartiallyClustered) case PartitioningCollection(partitionings) => PartitioningCollection(partitionings.map(resolvePartitioning(_, plan))) case RangePartitioning(ordering, numPartitions) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 8a65cb623f6e6..049270b505376 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -94,13 +94,13 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { checkQueryPlan(df, catalystDistribution, physical.KeyGroupedPartitioning(catalystDistribution.clustering, projectedPositions, - partitionValues, partitionValues)) + partitionValues, partitionValues, isPartiallyClustered = false)) // multiple group keys should work too as long as partition keys are subset of them df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY id, ts") checkQueryPlan(df, catalystDistribution, physical.KeyGroupedPartitioning(catalystDistribution.clustering, projectedPositions, - partitionValues, partitionValues)) + partitionValues, partitionValues, isPartiallyClustered = false)) } test("non-clustered distribution: no partition") { @@ -2892,4 +2892,124 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { Row("ccc", 30, 400.50))) } } + + test("SPARK-55848: dropDuplicates after SPJ with partial clustering should produce " + + "correct results") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(1, 50.0, cast('2020-01-02' as timestamp)), " + + "(2, 11.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + withSQLConf( + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> true.toString) { + // dropDuplicates on the join key after a partially-clustered SPJ must still + // produce the correct number of distinct ids. Before the fix, the + // partially-clustered partitioning was incorrectly treated as satisfying + // ClusteredDistribution, so EnsureRequirements did not insert an Exchange + // before the dedup, leading to duplicate rows. + val df = sql( + s""" + |SELECT DISTINCT i.id + |FROM testcat.ns.$items i + |JOIN testcat.ns.$purchases p ON i.id = p.item_id + |""".stripMargin) + checkAnswer(df, Seq(Row(1), Row(2), Row(3))) + + val allShuffles = collectAllShuffles(df.queryExecution.executedPlan) + assert(allShuffles.nonEmpty, + "should contain a shuffle for the post-join dedup with partial clustering") + } + } + + test("SPARK-55848: Window dedup after SPJ with partial clustering should produce " + + "correct results") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(1, 50.0, cast('2020-01-02' as timestamp)), " + + "(2, 11.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + withSQLConf( + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> true.toString) { + // Use ROW_NUMBER() OVER to dedup joined rows per id after a partially-clustered + // SPJ. The WINDOW operator requires ClusteredDistribution on i.id; with partial + // clustering the plan must insert a shuffle so that the window + // produces exactly one row per id. + val df = sql( + s""" + |SELECT id, price FROM ( + | SELECT i.id, i.price, + | ROW_NUMBER() OVER (PARTITION BY i.id ORDER BY i.price DESC) AS rn + | FROM testcat.ns.$items i + | JOIN testcat.ns.$purchases p ON i.id = p.item_id + |) t WHERE rn = 1 + |""".stripMargin) + checkAnswer(df, Seq(Row(1, 41.0f), Row(2, 10.0f), Row(3, 15.5f))) + + val allShuffles = collectAllShuffles(df.queryExecution.executedPlan) + assert(allShuffles.nonEmpty, + "should contain a shuffle for the post-join window with partial clustering") + } + } + + test("SPARK-55848: checkpointed scan with partial clustering and dedup") { + withTempDir { dir => + spark.sparkContext.setCheckpointDir(dir.getPath) + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(1, 50.0, cast('2020-01-02' as timestamp)), " + + "(2, 11.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> true.toString) { + // Checkpoint the items scan, then join with purchases under partial clustering, + // and finally dedup. The isPartiallyClustered flag must cause satisfies0()=false + // for ClusteredDistribution, so the dedup still produces correct distinct ids. + val itemsDf = spark.read.table(s"testcat.ns.$items").checkpoint() + val purchasesDf = spark.read.table(s"testcat.ns.$purchases") + + val df = itemsDf.as("i") + .join(purchasesDf.as("p"), col("i.id") === col("p.item_id")) + .select(col("i.id")).distinct() + checkAnswer(df, Seq(Row(1), Row(2), Row(3))) + + val allShuffles = collectAllShuffles(df.queryExecution.executedPlan) + assert(allShuffles.nonEmpty, + "should contain a shuffle for the post-join dedup with partial clustering") + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index 1cc0d795d74f8..ee852da536768 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -1129,7 +1129,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { EnsureRequirements.apply(smjExec) match { case ShuffledHashJoinExec(_, _, _, _, _, DummySparkPlan(_, _, left: KeyGroupedPartitioning, _, _), - ShuffleExchangeExec(KeyGroupedPartitioning(attrs, 4, pv, _), + ShuffleExchangeExec(KeyGroupedPartitioning(attrs, 4, pv, _, _), DummySparkPlan(_, _, SinglePartition, _, _), _, _), _) => assert(left.expressions == a1 :: Nil) assert(attrs == a1 :: Nil)

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b GIT binary patch literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q literal 0 HcmV?d00001 From b300b17d8c2d2f2ebc9df99e0acee01e1a755d79 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 11:51:59 +0000 Subject: [PATCH 343/400] Preparing development version 4.1.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 0b0b67f74f48b..d72a6a562432f 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.1 +Version: 4.1.2 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 1015e44593c57..f2afb76fef9ed 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 7468310695a6b..d2d9521f73834 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index ec57f5ecac8c4..e5dc64c542f5b 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 4e376a5127e07..eb8c7817c8ef0 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 899d41eb068ae..abc36aaa92b0c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index d8277cc5b4b87..2c4460fac9637 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6b4e8e48efc1e..426ff50535b76 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e5ddfcaa6b5ca..cc73ba82b29f1 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index 6ebf9ebb24a1c..f0486ebe8f215 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 5542f1901d388..45f640a406784 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index 9aa3ece76f89f..bf200867a41d2 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index c67fce0067d2b..da4366424525a 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index fff6c56714f03..d89a9f50f4626 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 3cfa9786b04e6..600a9c34769cf 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index 1243e161e89aa..c5b85f4027bc2 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 12779e9dd2ba4..cf092cb94a9e9 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index be6e03719776b..b47c502cadf1f 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 97f3c4e8c6d04..2f43af3eff918 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index e8bf39701aa18..d7eaf46ea110c 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 5f7ba112b9d3d..dcf6efff18a9d 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 98d1c0ec4a55c..d65da3de461c2 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 457a9f3bd733e..606ab7f01c964 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 0b7b7b32e8dd8..55cd208f70772 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 4b3de4e977445..bd5c50b8ace1c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.1 -SPARK_VERSION_SHORT: 4.1.1 +SPARK_VERSION: 4.1.2-SNAPSHOT +SPARK_VERSION_SHORT: 4.1.2 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.1"] + 'facetFilters': ["version:4.1.2"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index e1b54a02e8617..2e863dac54a09 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 35cb3da6b8c16..4448d976179fc 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 7563f4ae89876..390bab68a82de 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index fd914c74ca493..c59360214a178 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1a3574baf0837..5b2e4512e5f23 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 68e710fb9b659..80bf3b4053fe6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index dc757d788121c..f6eabc8c37c6c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 0a06a60816bd0..8ee776a66e67d 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.1" +__version__: str = "4.1.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 2e5d28c3f797e..d83963811474c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 388d2d56de408..9f8980b174e48 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 61d5f97149e1e..f69f81259355f 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 04f4dd893ebbb..f46cf6d31392a 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 2b3084ff34721..b43610317e1c1 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ab95d58e495d5..5fec0441d49f2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index d2f458f8b9fcc..c84ae04d3d735 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 69ebb70b48ede..f939c328c3cb2 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 268bfb9a37af3..0fddea5d79cbc 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index e38a39859c625..25394b1c6cb88 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 83a42dc2780b6..d4ee58e87c352 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d2d07a08aa9f8..285ea9ae4205c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 8c92db25a7539..0fec584548f67 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 4a63c5856adab..d854c0357657e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 1208cf822b1b9..ea01b79f8e289 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index 08fe9d65d7829..ace0daa15d772 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e79faee016fa6..c712edac1445f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml From 75f64af4e3402d6009ee6d9f266b1566c737d613 Mon Sep 17 00:00:00 2001 From: Garland Zhang Date: Mon, 5 Jan 2026 10:16:03 -0400 Subject: [PATCH 344/400] [SPARK-54886] Add base session created in SparkConnectService MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR makes SparkConnectService rely on its own SparkSession that is private and only intended for copying session configs to create new Sessions ### Why are the changes needed? The default session can get cleaned up in which case the SparkConnectService cannot recover as session creation fails on subsequent rpcs ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added basic testing ### Was this patch authored or co-authored using generative AI tooling? Yes Closes #52895 from garlandz-db/isolated_root_session. Authored-by: Garland Zhang Signed-off-by: Herman van Hövell (cherry picked from commit 0b5e92b7815db9e5ea3c5faf122fe2ceccfd5246) Signed-off-by: Herman van Hövell --- .../connect/service/SparkConnectService.scala | 1 + .../service/SparkConnectSessionManager.scala | 21 ++++++-- .../planner/SparkConnectServiceSuite.scala | 6 +++ .../ArtifactStatusesHandlerSuite.scala | 6 +++ .../SparkConnectCloneSessionSuite.scala | 1 + .../SparkConnectSessionManagerSuite.scala | 49 +++++++++++++++++++ 6 files changed, 80 insertions(+), 4 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 13ce2d64256b4..4641bc0a11067 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -436,6 +436,7 @@ object SparkConnectService extends Logging { return } + sessionManager.initializeBaseSession(sc) startGRPCService() createListenerAndUI(sc) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala index f28af0379a04c..6c468ba46cc03 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import com.google.common.cache.CacheBuilder -import org.apache.spark.{SparkEnv, SparkSQLException} +import org.apache.spark.{SparkContext, SparkEnv, SparkSQLException} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{INTERVAL, SESSION_HOLD_INFO} import org.apache.spark.sql.classic.SparkSession @@ -39,6 +39,9 @@ import org.apache.spark.util.ThreadUtils */ class SparkConnectSessionManager extends Logging { + // Base SparkSession created from the SparkContext, used to create new isolated sessions + @volatile private var baseSession: Option[SparkSession] = None + private val sessionStore: ConcurrentMap[SessionKey, SessionHolder] = new ConcurrentHashMap[SessionKey, SessionHolder]() @@ -48,6 +51,16 @@ class SparkConnectSessionManager extends Logging { .maximumSize(SparkEnv.get.conf.get(CONNECT_SESSION_MANAGER_CLOSED_SESSIONS_TOMBSTONES_SIZE)) .build[SessionKey, SessionHolderInfo]() + /** + * Initialize the base SparkSession from the provided SparkContext. This should be called once + * during SparkConnectService startup. + */ + def initializeBaseSession(sc: SparkContext): Unit = { + if (baseSession.isEmpty) { + baseSession = Some(SparkSession.builder().sparkContext(sc).getOrCreate().newSession()) + } + } + /** Executor for the periodic maintenance */ private val scheduledExecutor: AtomicReference[ScheduledExecutorService] = new AtomicReference[ScheduledExecutorService]() @@ -333,12 +346,12 @@ class SparkConnectSessionManager extends Logging { } private def newIsolatedSession(): SparkSession = { - val active = SparkSession.active - if (active.sparkContext.isStopped) { + val session = baseSession.get + if (session.sparkContext.isStopped) { assert(SparkSession.getDefaultSession.nonEmpty) SparkSession.getDefaultSession.get.newSession() } else { - active.newSession() + session.newSession() } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala index 2989471d36a03..6df50b6588c24 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala @@ -65,6 +65,12 @@ class SparkConnectServiceSuite with Logging with SparkConnectPlanTest { + override def beforeEach(): Unit = { + super.beforeEach() + SparkConnectService.sessionManager.invalidateAllSessions() + SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) + } + private def sparkSessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark) private def DEFAULT_UUID = UUID.fromString("89ea6117-1f45-4c03-ae27-f47c6aded093") diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala index 7ce3ff46f5537..caa71c644e6ac 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala @@ -42,6 +42,12 @@ class ArtifactStatusesHandlerSuite extends SharedSparkSession with ResourceHelpe val sessionId = UUID.randomUUID().toString + override def beforeEach(): Unit = { + super.beforeEach() + SparkConnectService.sessionManager.invalidateAllSessions() + SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) + } + def getStatuses(names: Seq[String], exist: Set[String]): ArtifactStatusesResponse = { val promise = Promise[ArtifactStatusesResponse]() val handler = new SparkConnectArtifactStatusesHandler(new DummyStreamObserver(promise)) { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala index 922c239526f31..42541b8c5f002 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala @@ -29,6 +29,7 @@ class SparkConnectCloneSessionSuite extends SharedSparkSession with BeforeAndAft override def beforeEach(): Unit = { super.beforeEach() SparkConnectService.sessionManager.invalidateAllSessions() + SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) } test("clone session with invalid target session ID format") { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala index 94deb83f6ad43..04d16a910746b 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkSQLException +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.pipelines.graph.{DataflowGraph, PipelineUpdateContextImpl} import org.apache.spark.sql.pipelines.logging.PipelineEvent import org.apache.spark.sql.test.SharedSparkSession @@ -32,6 +33,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA override def beforeEach(): Unit = { super.beforeEach() SparkConnectService.sessionManager.invalidateAllSessions() + SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) } test("sessionId needs to be an UUID") { @@ -171,4 +173,51 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA sessionHolder.getPipelineExecution(graphId).isEmpty, "pipeline execution was not removed") } + + test("baseSession allows creating sessions after default session is cleared") { + // Create a new session manager to test initialization + val sessionManager = new SparkConnectSessionManager() + + // Initialize the base session with the test SparkContext + sessionManager.initializeBaseSession(spark.sparkContext) + + // Clear the default and active sessions to simulate the scenario where + // SparkSession.active or SparkSession.getDefaultSession would fail + SparkSession.clearDefaultSession() + SparkSession.clearActiveSession() + + // Create an isolated session - this should still work because we have baseSession + val key = SessionKey("user", UUID.randomUUID().toString) + val sessionHolder = sessionManager.getOrCreateIsolatedSession(key, None) + + // Verify the session was created successfully + assert(sessionHolder != null) + assert(sessionHolder.session != null) + + // Clean up + sessionManager.closeSession(key) + } + + test("initializeBaseSession is idempotent") { + // Create a new session manager to test initialization + val sessionManager = new SparkConnectSessionManager() + + // Initialize the base session multiple times + sessionManager.initializeBaseSession(spark.sparkContext) + val key1 = SessionKey("user1", UUID.randomUUID().toString) + val sessionHolder1 = sessionManager.getOrCreateIsolatedSession(key1, None) + val baseSessionUUID1 = sessionHolder1.session.sessionUUID + + // Initialize again - should not change the base session + sessionManager.initializeBaseSession(spark.sparkContext) + val key2 = SessionKey("user2", UUID.randomUUID().toString) + val sessionHolder2 = sessionManager.getOrCreateIsolatedSession(key2, None) + + // Both sessions should be isolated from each other + assert(sessionHolder1.session.sessionUUID != sessionHolder2.session.sessionUUID) + + // Clean up + sessionManager.closeSession(key1) + sessionManager.closeSession(key2) + } } From 8e384977f04288130c116b7df74141ab13086d88 Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Wed, 7 Jan 2026 16:01:19 +0800 Subject: [PATCH 345/400] [SPARK-49872][FOLLOWUP] Remove Jackson JSON string length limit in KVStoreScalaSerializer MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Along the same lines as https://github.com/apache/spark/pull/52049, apply the same fix to `KVStoreScalaSerializer` as well. ### Why are the changes needed? The previous fix in https://github.com/apache/spark/pull/52049 only removed the Jackson JSON string length limit in `JsonProtocol`, which covered Spark event <=> JSON SerDe. However, the exact same problem exists in `KVStoreScalaSerializer` as well, which is triggered when SHS is configured to use LevelDB/RocksDB KVStores. Example call chain that triggers the issue: ``` ExecutionPage.render() → sqlStore.planGraph(executionId, version) → store.read(classOf[SparkPlanGraphWrapper], ...) → KVStoreSerializer.deserialize() → mapper.readValue() // ← FAILS HERE with 20MB limit ``` ### Does this PR introduce _any_ user-facing change? Yes, users won't hit the Jackson JSON string length limit any more when using SHS. ### How was this patch tested? Manually tested with a SQL workload that had an intentionally long string for the plan graph. The fix mechanism itself is tested by previous PRs https://github.com/apache/spark/pull/49163 and https://github.com/apache/spark/pull/52049, this is just applying the same fix to SHS as well. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53711 from rednaxelafx/fix-jackson-strlenlimit-kvstore. Authored-by: Kris Mok Signed-off-by: Wenchen Fan (cherry picked from commit ed201397c2ac3d9143bfb08508bbe7b5fba5f5cd) Signed-off-by: Wenchen Fan --- core/src/main/scala/org/apache/spark/status/KVUtils.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala index 1a9d36f6c6473..f9ea732f95140 100644 --- a/core/src/main/scala/org/apache/spark/status/KVUtils.scala +++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala @@ -25,6 +25,7 @@ import scala.jdk.CollectionConverters._ import scala.reflect.{classTag, ClassTag} import com.fasterxml.jackson.annotation.JsonInclude +import com.fasterxml.jackson.core.StreamReadConstraints import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.fusesource.leveldbjni.internal.NativeDB import org.rocksdb.RocksDBException @@ -76,6 +77,10 @@ private[spark] object KVUtils extends Logging { mapper.registerModule(DefaultScalaModule) mapper.setDefaultPropertyInclusion(JsonInclude.Include.NON_ABSENT) + // SPARK-49872: Remove jackson JSON string length limitation. + mapper.getFactory.setStreamReadConstraints( + StreamReadConstraints.builder().maxStringLength(Int.MaxValue).build() + ) } /** From 2dd054775bdac3fbbc505eccf0841946eae02f6d Mon Sep 17 00:00:00 2001 From: ilicmarkodb Date: Wed, 7 Jan 2026 16:11:23 +0800 Subject: [PATCH 346/400] [SPARK-54852][SQL] `NOT IN` subquery returns incorrect results with a collated table ### What changes were proposed in this pull request? ``` create or replace table t1 (c1 string collate utf8_lcase_rtrim); create or replace table t2 (c1 string collate utf8_lcase_rtrim); insert into t1 values ('a'); insert into t2 values ('A '); select * from t1 where c1 not in (select * from t2); -- should return no data, but it returns one row ``` When performing a hash join on collated columns, we first wrap the column with `CollationKey` during analysis. This is because the hash of `CollationKey` is collation-aware. The problem with this query is that there is no join during the analysis phase (we have `NOT IN`), and the join is added during the optimization phase. As a result, the join operates on raw columns, which are not collation-aware. This PR fixes the issue by rewriting the join keys in `HashJoin` trait. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53622 from ilicmarkodb/fix_not_in. Authored-by: ilicmarkodb Signed-off-by: Wenchen Fan (cherry picked from commit 57b2a065c41868d442ea4f7357f13268a9938302) Signed-off-by: Wenchen Fan --- .../catalyst/expressions/CollationKey.scala | 61 +++++++++++++- .../joins/BroadcastHashJoinExec.scala | 28 ++++++- .../spark/sql/execution/joins/HashJoin.scala | 16 ++++ .../joins/ShuffledHashJoinExec.scala | 26 +++++- .../spark/sql/collation/CollationSuite.scala | 81 +++++++++++++++++++ .../execution/joins/BroadcastJoinSuite.scala | 4 +- 6 files changed, 210 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala index 5d2fd14eee298..9a0aaea75f810 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.util.CollationFactory +import org.apache.spark.sql.catalyst.util.{CollationFactory, UnsafeRowUtils} import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.ArrayImplicits.SparkArrayOps case class CollationKey(expr: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = @@ -46,3 +47,61 @@ case class CollationKey(expr: Expression) extends UnaryExpression with ExpectsIn override def child: Expression = expr } + +object CollationKey { + /** + * Recursively process the expression in order to recursively replace non-binary collated strings + * with their associated collation key. + */ + def injectCollationKey(expr: Expression): Expression = { + injectCollationKey(expr, expr.dataType) + } + + private def injectCollationKey(expr: Expression, dt: DataType): Expression = { + dt match { + // For binary stable expressions, no special handling is needed. + case _ if UnsafeRowUtils.isBinaryStable(dt) => + expr + + // Inject CollationKey for non-binary collated strings. + case _: StringType => + CollationKey(expr) + + // Recursively process struct fields for non-binary structs. + case StructType(fields) => + val transformed = fields.zipWithIndex.map { case (f, i) => + val originalField = GetStructField(expr, i, Some(f.name)) + val injected = injectCollationKey(originalField, f.dataType) + (f, injected, injected.fastEquals(originalField)) + } + val anyChanged = transformed.exists { case (_, _, same) => !same } + if (!anyChanged) { + expr + } else { + val struct = CreateNamedStruct( + transformed.flatMap { case (f, injected, _) => + Seq(Literal(f.name), injected) + }.toImmutableArraySeq) + if (expr.nullable) { + If(IsNull(expr), Literal(null, struct.dataType), struct) + } else { + struct + } + } + + // Recursively process array elements for non-binary arrays. + case ArrayType(et, containsNull) => + val param: NamedExpression = NamedLambdaVariable("a", et, containsNull) + val funcBody: Expression = injectCollationKey(param, et) + if (!funcBody.fastEquals(param)) { + ArrayTransform(expr, LambdaFunction(funcBody, Seq(param))) + } else { + expr + } + + // Joins are not supported on maps, so there's no special handling for MapType. + case _ => + expr + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index b62d8f0798b6a..944ee3b059092 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, HashPartitioningLike, Partitioning, PartitioningCollection, UnspecifiedDistribution} import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan} @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * broadcast relation. This data is then placed in a Spark broadcast variable. The streamed * relation is not shuffled. */ -case class BroadcastHashJoinExec( +case class BroadcastHashJoinExec private( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -245,3 +245,27 @@ case class BroadcastHashJoinExec( newLeft: SparkPlan, newRight: SparkPlan): BroadcastHashJoinExec = copy(left = newLeft, right = newRight) } + +object BroadcastHashJoinExec extends JoinSelectionHelper { + def apply( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + buildSide: BuildSide, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan, + isNullAwareAntiJoin: Boolean = false): BroadcastHashJoinExec = { + val (normalizedLeftKeys, normalizedRightKeys) = HashJoin.normalizeJoinKeys(leftKeys, rightKeys) + + new BroadcastHashJoinExec( + normalizedLeftKeys, + normalizedRightKeys, + joinType, + buildSide, + condition, + left, + right, + isNullAwareAntiJoin) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index a1abb64e262df..fab14dba444dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.util.UnsafeRowUtils import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{CodegenSupport, ExplainUtils, RowIterator} import org.apache.spark.sql.execution.metric.SQLMetric @@ -41,6 +42,9 @@ private[joins] case class HashedRelationInfo( isEmpty: Boolean) trait HashJoin extends JoinCodegenSupport { + assert(leftKeys.forall(key => UnsafeRowUtils.isBinaryStable(key.dataType))) + assert(rightKeys.forall(key => UnsafeRowUtils.isBinaryStable(key.dataType))) + def buildSide: BuildSide override def simpleStringWithNodeId(): String = { @@ -724,6 +728,18 @@ trait HashJoin extends JoinCodegenSupport { object HashJoin extends CastSupport with SQLConfHelper { + /** + * Normalize join keys by injecting `CollationKey` when the keys are collated. + */ + def normalizeJoinKeys( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + ( + leftKeys.map(CollationKey.injectCollationKey), + rightKeys.map(CollationKey.injectCollationKey) + ) + } + private def canRewriteAsLongType(keys: Seq[Expression]): Boolean = { // TODO: support BooleanType, DateType and TimestampType keys.forall(_.dataType.isInstanceOf[IntegralType]) && diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 97ca74aee30c0..0f90f443ad41d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet} /** * Performs a hash join of two child relations by first shuffling the data using the join keys. */ -case class ShuffledHashJoinExec( +case class ShuffledHashJoinExec private ( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -659,3 +659,27 @@ case class ShuffledHashJoinExec( newLeft: SparkPlan, newRight: SparkPlan): ShuffledHashJoinExec = copy(left = newLeft, right = newRight) } + +object ShuffledHashJoinExec { + def apply( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + buildSide: BuildSide, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan, + isSkewJoin: Boolean = false): ShuffledHashJoinExec = { + val (normalizedLeftKeys, normalizedRightKeys) = HashJoin.normalizeJoinKeys(leftKeys, rightKeys) + + new ShuffledHashJoinExec( + normalizedLeftKeys, + normalizedRightKeys, + joinType, + buildSide, + condition, + left, + right, + isSkewJoin) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala index 6cdf681d65ca3..c84647066f25d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala @@ -2114,4 +2114,85 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"CREATE TABLE t (c STRING COLLATE system.builtin.UTF8_LCASE)") } } + + test("null aware anti join from NOT IN with collated columns") { + val expectedAnswer = Seq() + val (tableName1, tableName2) = ("t1", "t2") + withTable(tableName1, tableName2) { + sql(s"CREATE TABLE $tableName1 (C1 STRING COLLATE UTF8_LCASE_RTRIM)") + sql(s"CREATE TABLE $tableName2 (C1 STRING COLLATE UTF8_LCASE_RTRIM)") + sql(s"INSERT INTO $tableName1 VALUES ('a')") + sql(s"INSERT INTO $tableName2 VALUES ('A ')") + + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer) + + sql(s"INSERT INTO $tableName1 VALUES (NULL)") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer) + + sql(s"INSERT INTO $tableName1 VALUES ('b')") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer ++ Seq(Row("b"))) + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)" + + s" AND C1 = 'B '"), Row("b")) + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)" + + s" AND C1 > 'b'"), Seq()) + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)" + + s" AND C1 = 'c'"), Seq()) + + // This case results in empty output due to NULL in the t2. + sql(s"INSERT INTO $tableName2 VALUES (NULL)") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + Seq()) + } + } + + test("null aware anti join from NOT IN with collated columns in array type") { + val expectedAnswer = Seq() + val (tableName1, tableName2) = ("t1", "t2") + withTable(tableName1, tableName2) { + sql(s"CREATE TABLE $tableName1 (C1 ARRAY)") + sql(s"CREATE TABLE $tableName2 (C1 ARRAY)") + sql(s"INSERT INTO $tableName1 VALUES (ARRAY('a ', 'Aa '))") + sql(s"INSERT INTO $tableName2 VALUES (ARRAY('A', 'aa'))") + + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer) + + sql(s"INSERT INTO $tableName1 VALUES (NULL)") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer) + + // This case results in empty output due to NULL in the t2. + sql(s"INSERT INTO $tableName2 VALUES (NULL)") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + Seq()) + } + } + + test("null aware anti join from NOT IN with collated columns in struct type") { + val expectedAnswer = Seq() + val (tableName1, tableName2) = ("t1", "t2") + withTable(tableName1, tableName2) { + sql(s"CREATE TABLE $tableName1 (C1 STRUCT)") + sql(s"CREATE TABLE $tableName2 (C1 STRUCT)") + sql(s"INSERT INTO $tableName1 VALUES (named_struct('x', 'a ', 'y', 'Aa '))") + sql(s"INSERT INTO $tableName2 VALUES (named_struct('x', 'A', 'y', 'aa'))") + + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer) + + sql(s"INSERT INTO $tableName1 VALUES (NULL)") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + expectedAnswer) + + // This case results in empty output due to NULL in the t2. + sql(s"INSERT INTO $tableName2 VALUES (NULL)") + checkAnswer(sql(s"SELECT * FROM $tableName1 WHERE C1 NOT IN (SELECT * FROM $tableName2)"), + Seq()) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 69dd04e07d551..9bd858608cb9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -397,8 +397,8 @@ abstract class BroadcastJoinSuiteBase extends QueryTest with SQLTestUtils } } - private val bh = BroadcastHashJoinExec.toString - private val bl = BroadcastNestedLoopJoinExec.toString + private val bh = classOf[BroadcastHashJoinExec].getSimpleName + private val bl = classOf[BroadcastNestedLoopJoinExec].getSimpleName private def assertJoinBuildSide(sqlStr: String, joinMethod: String, buildSide: BuildSide): Any = { val executedPlan = stripAQEPlan(sql(sqlStr).queryExecution.executedPlan) From 8f9b2102e00ef1c96a4f630252fb173f7c6787a5 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Wed, 7 Jan 2026 14:45:29 -0800 Subject: [PATCH 347/400] [SPARK-53991][SQL][TEST][FOLLOW-UP] Improve KLL quantile errors by removing internal details ### What changes were proposed in this pull request? This PR improves error messages from the new KLL quantile sketch functions added in https://github.com/apache/spark/pull/52800. ### Why are the changes needed? The previous error messages reported internal DataSketches library state which was not meaningful for end users of the SQL/DF functions in Apache Spark. ### Does this PR introduce _any_ user-facing change? Yes, error messages are improved. For example, before this change, we observed the following: ``` SELECT kll_sketch_get_rank_bigint(agg, 5) AS wrong_type FROM ( SELECT kll_sketch_agg_float(col1) AS agg FROM t_float_1_5_through_7_11 ) > For function `kll_sketch_get_rank_bigint`, invalid KLL sketch binary data: reqOffset: 40, reqLength: 56, (reqOff + reqLen): 96, allocSize: 60" ``` Now the error message becomes: ``` > "Invalid call to `kll_sketch_get_rank_bigint`; only valid KLL sketch buffers are supported as inputs (such as those produced by the `kll_sketch_agg` function)." ``` ### How was this patch tested? This PR updates golden file test coverage to show the improved error messages. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53702 from dtenedor/kll-quantiles-golden-files. Authored-by: Daniel Tenedorio Signed-off-by: Daniel Tenedorio (cherry picked from commit 9aea212dfd4fe8b092eb44237c15db9cfd170db1) Signed-off-by: Daniel Tenedorio --- .../resources/error/error-conditions.json | 12 ++--- python/pyspark/sql/functions/builtin.py | 6 +-- python/pyspark/sql/tests/test_functions.py | 6 +-- .../expressions/aggregate/kllAggregates.scala | 24 +++++----- .../catalyst/expressions/kllExpressions.scala | 48 +++++++++---------- .../sql/errors/QueryExecutionErrors.scala | 28 ++++------- .../sql-tests/results/kllquantiles.sql.out | 39 ++++++--------- 7 files changed, 69 insertions(+), 94 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index dfaf8425a1a0c..aa0f0a89f97c4 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4228,21 +4228,15 @@ ], "sqlState" : "42K0E" }, - "KLL_SKETCH_INCOMPATIBLE_MERGE" : { + "KLL_INVALID_INPUT_SKETCH_BUFFER" : { "message" : [ - "For function , cannot merge KLL sketches: " - ], - "sqlState" : "22000" - }, - "KLL_SKETCH_INVALID_INPUT" : { - "message" : [ - "For function , invalid KLL sketch binary data: " + "Invalid call to ; only valid KLL sketch buffers are supported as inputs (such as those produced by the `kll_sketch_agg` function)." ], "sqlState" : "22000" }, "KLL_SKETCH_INVALID_QUANTILE_RANGE" : { "message" : [ - "For function , the quantile value must be between 0.0 and 1.0 (inclusive), but got ." + "For function , the quantile value must be between 0.0 and 1.0 (inclusive)." ], "sqlState" : "22003" }, diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 12bac34289d60..5bb1b2d8b5efc 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -26717,7 +26717,7 @@ def kll_sketch_to_string_bigint(col: "ColumnOrName") -> Column: >>> df = spark.createDataFrame([1,2,3,4,5], "INT") >>> sketch_df = df.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) >>> result = sketch_df.select(sf.kll_sketch_to_string_bigint("sketch")).first()[0] - >>> "Kll" in result and "N" in result + >>> "kll" in result.lower() True """ fn = "kll_sketch_to_string_bigint" @@ -26747,7 +26747,7 @@ def kll_sketch_to_string_float(col: "ColumnOrName") -> Column: >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "FLOAT") >>> sketch_df = df.agg(sf.kll_sketch_agg_float("value").alias("sketch")) >>> result = sketch_df.select(sf.kll_sketch_to_string_float("sketch")).first()[0] - >>> "Kll" in result and "N" in result + >>> "kll" in result.lower() True """ fn = "kll_sketch_to_string_float" @@ -26777,7 +26777,7 @@ def kll_sketch_to_string_double(col: "ColumnOrName") -> Column: >>> df = spark.createDataFrame([1.0,2.0,3.0,4.0,5.0], "DOUBLE") >>> sketch_df = df.agg(sf.kll_sketch_agg_double("value").alias("sketch")) >>> result = sketch_df.select(sf.kll_sketch_to_string_double("sketch")).first()[0] - >>> "Kll" in result and "N" in result + >>> "kll" in result.lower() True """ fn = "kll_sketch_to_string_double" diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 23c895f5629e0..b6d4e3a0547b3 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -2151,7 +2151,7 @@ def test_kll_sketch_to_string_bigint(self): result = sketch_df.select(F.kll_sketch_to_string_bigint("sketch")).first()[0] self.assertIsNotNone(result) self.assertIsInstance(result, str) - self.assertIn("Kll", result) + self.assertIn("kll", result.lower()) def test_kll_sketch_get_n_bigint(self): """Test kll_sketch_get_n_bigint function""" @@ -2212,7 +2212,7 @@ def test_kll_sketch_float_variants(self): # Test to_string string_result = sketch_df.select(F.kll_sketch_to_string_float("sketch")).first()[0] - self.assertIn("Kll", string_result) + self.assertIn("kll", string_result.lower()) # Test get_n n = sketch_df.select(F.kll_sketch_get_n_float("sketch")).first()[0] @@ -2240,7 +2240,7 @@ def test_kll_sketch_double_variants(self): # Test to_string string_result = sketch_df.select(F.kll_sketch_to_string_double("sketch")).first()[0] - self.assertIn("Kll", string_result) + self.assertIn("kll", string_result.lower()) # Test get_n n = sketch_df.select(F.kll_sketch_get_n_double("sketch")).first()[0] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala index 5891155cf7537..e74b22219cf61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala @@ -151,8 +151,8 @@ case class KllSketchAggBigint( updateBuffer.merge(input) updateBuffer } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } @@ -167,8 +167,8 @@ case class KllSketchAggBigint( try { KllLongsSketch.heapify(Memory.wrap(buffer)) } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } else { this.createAggregationBuffer() @@ -287,8 +287,8 @@ case class KllSketchAggFloat( updateBuffer.merge(input) updateBuffer } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } @@ -303,8 +303,8 @@ case class KllSketchAggFloat( try { KllFloatsSketch.heapify(Memory.wrap(buffer)) } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } else { this.createAggregationBuffer() @@ -425,8 +425,8 @@ case class KllSketchAggDouble( updateBuffer.merge(input) updateBuffer } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } @@ -441,8 +441,8 @@ case class KllSketchAggDouble( try { KllDoublesSketch.heapify(Memory.wrap(buffer)) } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } else { this.createAggregationBuffer() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala index b446b2d0d4433..af6c1a32e229f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala @@ -50,8 +50,8 @@ case class KllSketchToStringBigint(child: Expression) extends KllSketchToStringB val sketch = KllLongsSketch.heapify(Memory.wrap(buffer)) UTF8String.fromString(sketch.toString()) } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -78,8 +78,8 @@ case class KllSketchToStringFloat(child: Expression) extends KllSketchToStringBa val sketch = KllFloatsSketch.heapify(Memory.wrap(buffer)) UTF8String.fromString(sketch.toString()) } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -106,8 +106,8 @@ case class KllSketchToStringDouble(child: Expression) extends KllSketchToStringB val sketch = KllDoublesSketch.heapify(Memory.wrap(buffer)) UTF8String.fromString(sketch.toString()) } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -144,8 +144,8 @@ case class KllSketchGetNBigint(child: Expression) extends KllSketchGetNBase { val sketch = KllLongsSketch.heapify(Memory.wrap(buffer)) sketch.getN() } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -172,8 +172,8 @@ case class KllSketchGetNFloat(child: Expression) extends KllSketchGetNBase { val sketch = KllFloatsSketch.heapify(Memory.wrap(buffer)) sketch.getN() } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -200,8 +200,8 @@ case class KllSketchGetNDouble(child: Expression) extends KllSketchGetNBase { val sketch = KllDoublesSketch.heapify(Memory.wrap(buffer)) sketch.getN() } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -241,8 +241,8 @@ case class KllSketchMergeBigint(left: Expression, right: Expression) extends Kll leftSketch.merge(rightSketch) leftSketch.toByteArray } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -272,8 +272,8 @@ case class KllSketchMergeFloat(left: Expression, right: Expression) extends KllS leftSketch.merge(rightSketch) leftSketch.toByteArray } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -303,8 +303,8 @@ case class KllSketchMergeDouble(left: Expression, right: Expression) extends Kll leftSketch.merge(rightSketch) leftSketch.toByteArray } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchIncompatibleMergeError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } } @@ -456,12 +456,12 @@ abstract class KllSketchGetQuantileBase } catch { case e: org.apache.datasketches.common.SketchesArgumentException => if (e.getMessage.contains("normalized rank")) { - throw QueryExecutionErrors.kllSketchInvalidQuantileRangeError(prettyName, rankForError) + throw QueryExecutionErrors.kllSketchInvalidQuantileRangeError(prettyName) } else { - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } @@ -617,8 +617,8 @@ abstract class KllSketchGetRankBase try { operation } catch { - case e: Exception => - throw QueryExecutionErrors.kllSketchInvalidInputError(prettyName, e.getMessage) + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 27aba1f7f2dfb..351868fcc2e29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -2830,6 +2830,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "function" -> toSQLId(function))) } + def kllInvalidInputSketchBuffer(function: String, reason: String = ""): Throwable = { + new SparkRuntimeException( + errorClass = "KLL_INVALID_INPUT_SKETCH_BUFFER", + messageParameters = Map( + "function" -> toSQLId(function))) + } + def hllUnionDifferentLgK(left: Int, right: Int, function: String): Throwable = { new SparkRuntimeException( errorClass = "HLL_UNION_DIFFERENT_LG_K", @@ -3195,28 +3202,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE messageParameters = Map("function" -> toSQLId(function))) } - def kllSketchInvalidQuantileRangeError(function: String, quantile: Double): Throwable = { + def kllSketchInvalidQuantileRangeError(function: String): Throwable = { new SparkRuntimeException( errorClass = "KLL_SKETCH_INVALID_QUANTILE_RANGE", messageParameters = Map( - "functionName" -> toSQLId(function), - "quantile" -> toSQLValue(quantile, DoubleType))) - } - - def kllSketchInvalidInputError(function: String, reason: String): Throwable = { - new SparkRuntimeException( - errorClass = "KLL_SKETCH_INVALID_INPUT", - messageParameters = Map( - "functionName" -> toSQLId(function), - "reason" -> reason)) - } - - def kllSketchIncompatibleMergeError(function: String, reason: String): Throwable = { - new SparkRuntimeException( - errorClass = "KLL_SKETCH_INCOMPATIBLE_MERGE", - messageParameters = Map( - "functionName" -> toSQLId(function), - "reason" -> reason)) + "functionName" -> toSQLId(function))) } def kllSketchKMustBeConstantError(function: String): Throwable = { diff --git a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out index 863bded1599b5..fef44781e17cc 100644 --- a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out @@ -837,8 +837,7 @@ org.apache.spark.SparkRuntimeException "errorClass" : "KLL_SKETCH_INVALID_QUANTILE_RANGE", "sqlState" : "22003", "messageParameters" : { - "functionName" : "`kll_sketch_get_quantile_bigint`", - "quantile" : "-0.5D" + "functionName" : "`kll_sketch_get_quantile_bigint`" } } @@ -857,8 +856,7 @@ org.apache.spark.SparkRuntimeException "errorClass" : "KLL_SKETCH_INVALID_QUANTILE_RANGE", "sqlState" : "22003", "messageParameters" : { - "functionName" : "`kll_sketch_get_quantile_bigint`", - "quantile" : "1.5D" + "functionName" : "`kll_sketch_get_quantile_bigint`" } } @@ -877,8 +875,7 @@ org.apache.spark.SparkRuntimeException "errorClass" : "KLL_SKETCH_INVALID_QUANTILE_RANGE", "sqlState" : "22003", "messageParameters" : { - "functionName" : "`kll_sketch_get_quantile_float`", - "quantile" : "-0.1D" + "functionName" : "`kll_sketch_get_quantile_float`" } } @@ -894,11 +891,10 @@ struct<> -- !query output org.apache.spark.SparkRuntimeException { - "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", "sqlState" : "22000", "messageParameters" : { - "functionName" : "`kll_sketch_get_rank_bigint`", - "reason" : "reqOffset: 40, reqLength: 56, (reqOff + reqLen): 96, allocSize: 60" + "function" : "`kll_sketch_get_rank_bigint`" } } @@ -915,11 +911,10 @@ struct<> -- !query output org.apache.spark.SparkRuntimeException { - "errorClass" : "KLL_SKETCH_INCOMPATIBLE_MERGE", + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", "sqlState" : "22000", "messageParameters" : { - "functionName" : "`kll_sketch_merge_bigint`", - "reason" : "reqOffset: 40, reqLength: 56, (reqOff + reqLen): 96, allocSize: 60" + "function" : "`kll_sketch_merge_bigint`" } } @@ -931,11 +926,10 @@ struct<> -- !query output org.apache.spark.SparkRuntimeException { - "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", "sqlState" : "22000", "messageParameters" : { - "functionName" : "`kll_sketch_get_quantile_bigint`", - "reason" : "Error combination of PreInts and SerVer: PreInts: 110, SerVer: 111" + "function" : "`kll_sketch_get_quantile_bigint`" } } @@ -1098,11 +1092,10 @@ struct<> -- !query output org.apache.spark.SparkRuntimeException { - "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", "sqlState" : "22000", "messageParameters" : { - "functionName" : "`kll_sketch_get_n_bigint`", - "reason" : "A sketch memory image must be at least 8 bytes. 4" + "function" : "`kll_sketch_get_n_bigint`" } } @@ -1114,11 +1107,10 @@ struct<> -- !query output org.apache.spark.SparkRuntimeException { - "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", "sqlState" : "22000", "messageParameters" : { - "functionName" : "`kll_sketch_get_n_float`", - "reason" : "A sketch memory image must be at least 8 bytes. 4" + "function" : "`kll_sketch_get_n_float`" } } @@ -1130,11 +1122,10 @@ struct<> -- !query output org.apache.spark.SparkRuntimeException { - "errorClass" : "KLL_SKETCH_INVALID_INPUT", + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", "sqlState" : "22000", "messageParameters" : { - "functionName" : "`kll_sketch_get_n_double`", - "reason" : "A sketch memory image must be at least 8 bytes. 4" + "function" : "`kll_sketch_get_n_double`" } } From 787f3fd2325b12570a7d47291a842ebac197fe57 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 9 Jan 2026 16:05:10 +0900 Subject: [PATCH 348/400] [SPARK-54982][INFRA] Use ASF_NEXUS_TOKEN to release and drop staging repository when finalizing the RC ### What changes were proposed in this pull request? This PR proposes to use ASF_NEXUS_TOKEN to release and drop staging repository when finalizing the RC. While I am here, I made the commands more portable/universal - in practice, I wanted to use the same commands when I test in my local. ### Why are the changes needed? To automatically release/drop the staging repo. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? I manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53744 from HyukjinKwon/minor-compat-command. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit c26a3e481e028961453525e065e30d93c94fd345) Signed-off-by: Hyukjin Kwon --- .github/workflows/publish_snapshot.yml | 1 + .github/workflows/release.yml | 10 ++++++-- dev/create-release/do-release-docker.sh | 1 + dev/create-release/release-build.sh | 34 +++++++++++++++---------- 4 files changed, 31 insertions(+), 15 deletions(-) diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index 6e2e5709bbd18..b20c34d8ec5a1 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -67,6 +67,7 @@ jobs: env: ASF_USERNAME: ${{ secrets.NEXUS_USER }} ASF_PASSWORD: ${{ secrets.NEXUS_PW }} + ASF_NEXUS_TOKEN: ${{ secrets.NEXUS_TOKEN }} GPG_KEY: "not_used" GPG_PASSPHRASE: "not_used" GIT_REF: ${{ matrix.branch }} diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 5de61c831cbef..ab10d04d9badd 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -28,6 +28,10 @@ # - ASF_PASSWORD: # The password associated with your ASF account. # +# - ASF_NEXUS_TOKEN: +# ASF Nexus API token associated with your ASF account. +# Can be found in https://repository.apache.org/#profile;User%20Token +# # - GPG_PRIVATE_KEY: # Your GPG private key, exported using: # gpg --armor --export-secret-keys ABCD1234 > private.key @@ -128,6 +132,7 @@ jobs: GIT_NAME: "${{ github.actor }}" ASF_USERNAME: "${{ secrets.ASF_USERNAME }}" ASF_PASSWORD: "${{ secrets.ASF_PASSWORD }}" + ASF_NEXUS_TOKEN: "${{ secrets.ASF_NEXUS_TOKEN }}" GPG_PRIVATE_KEY: "${{ secrets.GPG_PRIVATE_KEY }}" GPG_PASSPHRASE: "${{ secrets.GPG_PASSPHRASE }}" PYPI_API_TOKEN: "${{ secrets.PYPI_API_TOKEN }}" @@ -163,6 +168,7 @@ jobs: GPG_PRIVATE_KEY="not_used" GPG_PASSPHRASE="not_used" ASF_USERNAME="gurwls223" + ASF_NEXUS_TOKEN="not_used" export SKIP_TAG=1 unset RELEASE_VERSION else @@ -170,7 +176,7 @@ jobs: export DRYRUN_MODE=0 fi - export ASF_PASSWORD GPG_PRIVATE_KEY GPG_PASSPHRASE ASF_USERNAME + export ASF_PASSWORD GPG_PRIVATE_KEY GPG_PASSPHRASE ASF_USERNAME ASF_NEXUS_TOKEN export GIT_BRANCH="${GIT_BRANCH:-master}" [ -n "$RELEASE_VERSION" ] && export RELEASE_VERSION @@ -239,7 +245,7 @@ jobs: # Redact sensitive information in log files shopt -s globstar nullglob FILES=("$RELEASE_DIR/docker-build.log" "$OUTPUT_DIR/"*.log) - PATTERNS=("$ASF_USERNAME" "$ASF_PASSWORD" "$GPG_PRIVATE_KEY" "$GPG_PASSPHRASE" "$PYPI_API_TOKEN") + PATTERNS=("$ASF_USERNAME" "$ASF_PASSWORD" "$GPG_PRIVATE_KEY" "$GPG_PASSPHRASE" "$PYPI_API_TOKEN" "$ASF_NEXUS_TOKEN") for file in "${FILES[@]}"; do [ -f "$file" ] || continue cp "$file" "$file.bak" diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index eaa8073fbca6e..3a395e3c266bd 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -146,6 +146,7 @@ RELEASE_TAG=$RELEASE_TAG GIT_REF=$GIT_REF SPARK_PACKAGE_VERSION=$SPARK_PACKAGE_VERSION ASF_USERNAME=$ASF_USERNAME +ASF_NEXUS_TOKEN=$ASF_NEXUS_TOKEN GIT_NAME=$GIT_NAME GIT_EMAIL=$GIT_EMAIL GPG_KEY=$GPG_KEY diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 21f0d052047b3..9a83eab694607 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -40,6 +40,7 @@ SPARK_VERSION - (optional) Version of Spark being built (e.g. 2.1.2) ASF_USERNAME - Username of ASF committer account ASF_PASSWORD - Password of ASF committer account +ASF_NEXUS_TOKEN - API token in ASF Nexus reposiotry GPG_KEY - GPG key used to sign release artifacts GPG_PASSPHRASE - Passphrase for GPG key @@ -493,15 +494,18 @@ EOF echo "KEYS sync'ed" rm -rf svn-spark - # TODO: Test it in the actual release # Release artifacts in the Nexus repository # Find latest orgapachespark-* repo for this release version - REPO_ID=$(curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_PASSWORD" \ - https://repository.apache.org/service/local/staging/profile_repositories | \ - grep -A 13 "orgapachespark-" | \ - awk '// { id = $0 } // && $0 ~ /Apache Spark '"$RELEASE_VERSION"'/ { print id }' | \ - grep -oP '(?<=)orgapachespark-[0-9]+(?=)' | \ - sort -V | tail -n 1) + REPO_ID=$( + curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_NEXUS_TOKEN" \ + https://repository.apache.org/service/local/staging/profile_repositories | + grep -A 13 "orgapachespark-" | + awk '// { id = $0 } + // && $0 ~ /Apache Spark '"$RELEASE_VERSION"'/ { print id }' | + sed -n 's/.*\(orgapachespark-[0-9][0-9]*\)<\/repositoryId>.*/\1/p' | + sort -V | + tail -n 1 + ) if [[ -z "$REPO_ID" ]]; then echo "No matching staging repository found for Apache Spark $RELEASE_VERSION" @@ -511,7 +515,7 @@ EOF echo "Using repository ID: $REPO_ID" # Release the repository - curl --retry 10 --retry-all-errors -s -u "$APACHE_USERNAME:$APACHE_PASSWORD" \ + curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_NEXUS_TOKEN" \ -H "Content-Type: application/json" \ -X POST https://repository.apache.org/service/local/staging/bulk/promote \ -d "{\"data\": {\"stagedRepositoryIds\": [\"$REPO_ID\"], \"description\": \"Apache Spark $RELEASE_VERSION\"}}" @@ -519,9 +523,13 @@ EOF # Wait for release to complete echo "Waiting for release to complete..." while true; do - STATUS=$(curl --retry 10 --retry-all-errors -s -u "$APACHE_USERNAME:$APACHE_PASSWORD" \ - https://repository.apache.org/service/local/staging/repository/$REPO_ID | \ - grep -oPm1 "(?<=)[^<]+") + STATUS=$( + curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_NEXUS_TOKEN" \ + https://repository.apache.org/service/local/staging/repository/$REPO_ID | + sed -n 's:.*\([^<]*\).*:\1:p' | + head -n 1 + ) + echo "Current state: $STATUS" if [[ "$STATUS" == "released" ]]; then echo "Release complete." @@ -537,7 +545,7 @@ EOF done # Drop the repository after release - curl --retry 10 --retry-all-errors -s -u "$APACHE_USERNAME:$APACHE_PASSWORD" \ + curl --retry 10 --retry-all-errors -s -u "$ASF_USERNAME:$ASF_NEXUS_TOKEN" \ -H "Content-Type: application/json" \ -X POST https://repository.apache.org/service/local/staging/bulk/drop \ -d "{\"data\": {\"stagedRepositoryIds\": [\"$REPO_ID\"], \"description\": \"Dropped after release\"}}" @@ -547,7 +555,7 @@ EOF # Remove old releases from the mirror # Extract major.minor prefix RELEASE_SERIES=$(echo "$RELEASE_VERSION" | cut -d. -f1-2) - + # Fetch existing dist URLs OLD_VERSION=$(svn ls https://dist.apache.org/repos/dist/release/spark/ | \ grep "^spark-$RELEASE_SERIES" | \ From ede112e850167b922c2567b81a8ed7a872214dac Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Sun, 4 Jan 2026 08:49:41 -0800 Subject: [PATCH 349/400] [SPARK-54610][SDP][DOCS] Improve SDP Programming Guide ### What changes were proposed in this pull request? Improved the SDP Programming Guide ### Why are the changes needed? Better docs ### Does this PR introduce _any_ user-facing change? Yes, given that the changes are user docs-related. ### How was this patch tested? Reviewed locally ### Was this patch authored or co-authored using generative AI tooling? No (unless auto-completion in IntelliJ IDEA counts) Closes #53346 from jaceklaskowski/SPARK-54610-sdp-programming-guide. Authored-by: Jacek Laskowski Signed-off-by: Sandy Ryza (cherry picked from commit 2f12bc6558757fae41bf1fccf4f342fa78541e82) Signed-off-by: Sandy Ryza --- ...declarative-pipelines-programming-guide.md | 199 +++++++++++------- 1 file changed, 123 insertions(+), 76 deletions(-) diff --git a/docs/declarative-pipelines-programming-guide.md b/docs/declarative-pipelines-programming-guide.md index 0ca04c644f1ba..859258a430dd6 100644 --- a/docs/declarative-pipelines-programming-guide.md +++ b/docs/declarative-pipelines-programming-guide.md @@ -24,7 +24,7 @@ license: | ## What is Spark Declarative Pipelines (SDP)? -Spark Declarative Pipelines (SDP) is a declarative framework for building reliable, maintainable, and testable data pipelines on Spark. SDP simplifies ETL development by allowing you to focus on the transformations you want to apply to your data, rather than the mechanics of pipeline execution. +Spark Declarative Pipelines (SDP) is a declarative framework for building reliable, maintainable, and testable data pipelines on Apache Spark. SDP simplifies ETL development by allowing you to focus on the transformations you want to apply to your data, rather than the mechanics of pipeline execution. SDP is designed for both batch and streaming data processing, supporting common use cases such as: - Data ingestion from cloud storage (Amazon S3, Azure ADLS Gen2, Google Cloud Storage) @@ -62,10 +62,10 @@ SDP creates the table named `target_table` along with a flow that reads new data ### Datasets -A dataset is queryable object that's the output of one of more flows within a pipeline. Flows in the pipeline can also read from datasets produced in the pipeline. +A dataset is a queryable object that's the output of one of more flows within a pipeline. Flows in the pipeline can also read from datasets produced in the pipeline. - **Streaming Table** – a definition of a table and one or more streaming flows written into it. Streaming tables support incremental processing of data, allowing you to process only new data as it arrives. -- **Materialized View** – is a view that is precomputed into a table. A materialized view always has exactly one batch flow writing to it. +- **Materialized View** – a view that is precomputed into a table. A materialized view always has exactly one batch flow writing to it. - **Temporary View** – a view that is scoped to an execution of the pipeline. It can be referenced from flows within the pipeline. It's useful for encapsulating transformations and intermediate logical entities that multiple other elements of the pipeline depend on. ### Pipelines @@ -74,11 +74,15 @@ A pipeline is the primary unit of development and execution in SDP. A pipeline c ### Pipeline Projects -A pipeline project is a set of source files that contain code that define the datasets and flows that make up a pipeline. These source files can be `.py` or `.sql` files. +A pipeline project is a set of source files that contain code definitions of the datasets and flows that make up a pipeline. The source files can be `.py` or `.sql` files. -A YAML-formatted pipeline spec file contains the top-level configuration for the pipeline project. It supports the following fields: -- **libraries** (Required) - Paths where source files can be found. -- **storage** (Required) – A directory where checkpoints can be stored for streams within the pipeline. +It's conventional to name pipeline spec files `spark-pipeline.yml` or `spark-pipeline.yaml`. + +A YAML-formatted pipeline spec file contains the top-level configuration for the pipeline project with the following fields: + +- **name** (Required) - The name of the pipeline project. +- **libraries** (Required) - The paths with the transformation source files in SQL or Python. +- **storage** (Required) – A directory where checkpoints can be stored for streaming tables within the pipeline. - **database** (Optional) - The default target database for pipeline outputs. **schema** can alternatively be used as an alias. - **catalog** (Optional) - The default target catalog for pipeline outputs. - **configuration** (Optional) - Map of Spark configuration properties. @@ -90,175 +94,209 @@ name: my_pipeline libraries: - glob: include: transformations/** +storage: file:///absolute/path/to/storage/dir catalog: my_catalog database: my_db configuration: spark.sql.shuffle.partitions: "1000" ``` -It's conventional to name pipeline spec files `spark-pipeline.yml`. - The `spark-pipelines init` command, described below, makes it easy to generate a pipeline project with default configuration and directory structure. - ## The `spark-pipelines` Command Line Interface -The `spark-pipelines` command line interface (CLI) is the primary way to execute a pipeline. It also contains an `init` subcommand for generating a pipeline project and a `dry-run` subcommand for validating a pipeline. +The `spark-pipelines` command line interface (CLI) is the primary way to manage a pipeline. `spark-pipelines` is built on top of `spark-submit`, meaning that it supports all cluster managers supported by `spark-submit`. It supports all `spark-submit` arguments except for `--class`. ### `spark-pipelines init` -`spark-pipelines init --name my_pipeline` generates a simple pipeline project, inside a directory named "my_pipeline", including a spec file and example definitions. +`spark-pipelines init --name my_pipeline` generates a simple pipeline project, inside a directory named `my_pipeline`, including a spec file and example transformation definitions. ### `spark-pipelines run` -`spark-pipelines run` launches an execution of a pipeline and monitors its progress until it completes. The `--spec` parameter allows selecting the pipeline spec file. If not provided, the CLI will look in the current directory and parent directories for a file named `spark-pipeline.yml` or `spark-pipeline.yaml`. +`spark-pipelines run` launches an execution of a pipeline and monitors its progress until it completes. + +The `--spec` parameter allows selecting the pipeline spec file. If not provided, the CLI will look in the current directory and parent directories for one of the files: + +* `spark-pipeline.yml` +* `spark-pipeline.yaml` ### `spark-pipelines dry-run` `spark-pipelines dry-run` launches an execution of a pipeline that doesn't write or read any data, but catches many kinds of errors that would be caught if the pipeline were to actually run. E.g. - Syntax errors – e.g. invalid Python or SQL code -- Analysis errors – e.g. selecting from a table that doesn't exist or selecting a column that doesn't exist +- Analysis errors – e.g. selecting from a table or a column that doesn't exist - Graph validation errors - e.g. cyclic dependencies ## Programming with SDP in Python -SDP Python functions are defined in the `pyspark.pipelines` module. Your pipelines implemented with the Python API must import this module. It's common to alias the module to `dp` to limit the number of characters you need to type when using its APIs. +SDP Python definitions are defined in the `pyspark.pipelines` module. + +Your pipelines implemented with the Python API must import this module. It's recommended to alias the module to `dp`. ```python from pyspark import pipelines as dp ``` -### Creating a Materialized View with Python +### Creating a Materialized View in Python -The `@dp.materialized_view` decorator tells SDP to create a materialized view based on the results returned by a function that performs a batch read: +The `@dp.materialized_view` decorator tells SDP to create a materialized view based on the results of a function that performs a batch read: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame @dp.materialized_view -def basic_mv(): +def basic_mv() -> DataFrame: return spark.table("samples.nyctaxi.trips") ``` -Optionally, you can specify the table name using the `name` argument: +The name of the materialized view is derived from the name of the function. + +You can specify the name of the materialized view using the `name` argument: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame @dp.materialized_view(name="trips_mv") -def basic_mv(): +def basic_mv() -> DataFrame: return spark.table("samples.nyctaxi.trips") ``` -### Creating a Temporary View with Python +### Creating a Temporary View in Python -The `@dp.temporary_view` decorator tells SDP to create a temporary view based on the results returned by a function that performs a batch read: +The `@dp.temporary_view` decorator tells SDP to create a temporary view based on the results of a function that performs a batch read: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame @dp.temporary_view -def basic_tv(): +def basic_tv() -> DataFrame: return spark.table("samples.nyctaxi.trips") ``` This temporary view can be read by other queries within the pipeline, but can't be read outside the scope of the pipeline. -### Creating a Streaming Table with Python +### Creating a Streaming Table in Python -Similarly, you can create a streaming table by using the `@dp.table` decorator with a function that performs a streaming read: +You can create a streaming table using the `@dp.table` decorator with a function that performs a streaming read: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame @dp.table -def basic_st(): +def basic_st() -> DataFrame: return spark.readStream.table("samples.nyctaxi.trips") ``` -### Loading Data from a Streaming Source +### Loading Data from Streaming Sources in Python + +SDP supports loading data from all the formats supported by Spark Structured Streaming (`spark.readStream`). -SDP supports loading data from all formats supported by Spark. For example, you can create a streaming table whose query reads from a Kafka topic: +For example, you can create a streaming table whose query reads from a Kafka topic: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame @dp.table -def ingestion_st(): +def ingestion_st() -> DataFrame: return ( - spark.readStream.format("kafka") + spark.readStream + .format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "orders") .load() ) ``` -For batch reads: +### Loading Data from Batch Sources in Python + +SDP supports loading data from all the formats supported by Spark SQL (`spark.read`). ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame @dp.materialized_view -def batch_mv(): +def batch_mv() -> DataFrame: return spark.read.format("json").load("/datasets/retail-org/sales_orders") ``` -### Querying Tables Defined in Your Pipeline +### Querying Tables Defined in a Pipeline in Python You can reference other tables defined in your pipeline in the same way you'd reference tables defined outside your pipeline: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame from pyspark.sql.functions import col @dp.table -def orders(): +def orders() -> DataFrame: return ( - spark.readStream.format("kafka") + spark.readStream + .format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "orders") .load() ) @dp.materialized_view -def customers(): - return spark.read.format("csv").option("header", True).load("/datasets/retail-org/customers") +def customers() -> DataFrame: + return ( + spark.read + .format("csv") + .option("header", True) + .load("/datasets/retail-org/customers") + ) @dp.materialized_view -def customer_orders(): - return (spark.table("orders") - .join(spark.table("customers"), "customer_id") - .select("customer_id", - "order_number", - "state", - col("order_datetime").cast("int").cast("timestamp").cast("date").alias("order_date"), +def customer_orders() -> DataFrame: + return ( + spark.table("orders") + .join( + spark.table("customers"), "customer_id") + .select( + "customer_id", + "order_number", + "state", + col("order_datetime").cast("date").alias("order_date"), + ) ) ) @dp.materialized_view -def daily_orders_by_state(): - return (spark.table("customer_orders") +def daily_orders_by_state() -> DataFrame: + return ( + spark.table("customer_orders") .groupBy("state", "order_date") - .count().withColumnRenamed("count", "order_count") + .count() + .withColumnRenamed("count", "order_count") ) ``` -### Creating Tables in a For Loop +### Creating Tables in For Loop in Python You can use Python `for` loops to create multiple tables programmatically: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame from pyspark.sql.functions import collect_list, col @dp.temporary_view() -def customer_orders(): +def customer_orders() -> DataFrame: orders = spark.table("samples.tpch.orders") customer = spark.table("samples.tpch.customer") - return (orders.join(customer, orders.o_custkey == customer.c_custkey) + return ( + orders + .join(customer, orders.o_custkey == customer.c_custkey) .select( col("c_custkey").alias("custkey"), col("c_name").alias("name"), @@ -267,19 +305,22 @@ def customer_orders(): col("o_orderkey").alias("orderkey"), col("o_orderstatus").alias("orderstatus"), col("o_totalprice").alias("totalprice"), - col("o_orderdate").alias("orderdate")) + col("o_orderdate").alias("orderdate"), + ) ) @dp.temporary_view() -def nation_region(): +def nation_region() -> DataFrame: nation = spark.table("samples.tpch.nation") region = spark.table("samples.tpch.region") - return (nation.join(region, nation.n_regionkey == region.r_regionkey) + return ( + nation + .join(region, nation.n_regionkey == region.r_regionkey) .select( col("n_name").alias("nation"), col("r_name").alias("region"), - col("n_nationkey").alias("nationkey") + col("n_nationkey").alias("nationkey"), ) ) @@ -289,11 +330,13 @@ region_list = spark.table("samples.tpch.region").select(collect_list("r_name")). # Iterate through region names to create new region-specific materialized views for region in region_list: @dp.table(name=f"{region.lower().replace(' ', '_')}_customer_orders") - def regional_customer_orders(region_filter=region): + def regional_customer_orders(region_filter=region) -> DataFrame: customer_orders = spark.table("customer_orders") nation_region = spark.table("nation_region") - return (customer_orders.join(nation_region, customer_orders.nationkey == nation_region.nationkey) + return ( + customer_orders + .join(nation_region, customer_orders.nationkey == nation_region.nationkey) .select( col("custkey"), col("name"), @@ -303,35 +346,37 @@ for region in region_list: col("orderkey"), col("orderstatus"), col("totalprice"), - col("orderdate") - ).filter(f"region = '{region_filter}'") + col("orderdate"), + ) + .filter(f"region = '{region_filter}'") ) ``` -### Using Multiple Flows to Write to a Single Target +### Using Multiple Flows to Write to a Single Target in Python -You can create multiple flows that append data to the same target: +You can create multiple flows that append data to the same dataset: ```python from pyspark import pipelines as dp +from pyspark.sql import DataFrame # create a streaming table dp.create_streaming_table("customers_us") -# add the first append flow +# define the first append flow @dp.append_flow(target = "customers_us") -def append1(): +def append_customers_us_west() -> DataFrame: return spark.readStream.table("customers_us_west") -# add the second append flow +# define the second append flow @dp.append_flow(target = "customers_us") -def append2(): +def append_customers_us_east() -> DataFrame: return spark.readStream.table("customers_us_east") ``` ## Programming with SDP in SQL -### Creating a Materialized View with SQL +### Creating a Materialized View in SQL The basic syntax for creating a materialized view with SQL is: @@ -340,7 +385,7 @@ CREATE MATERIALIZED VIEW basic_mv AS SELECT * FROM samples.nyctaxi.trips; ``` -### Creating a Temporary View with SQL +### Creating a Temporary View in SQL The basic syntax for creating a temporary view with SQL is: @@ -349,7 +394,7 @@ CREATE TEMPORARY VIEW basic_tv AS SELECT * FROM samples.nyctaxi.trips; ``` -### Creating a Streaming Table with SQL +### Creating a Streaming Table in SQL When creating a streaming table, use the `STREAM` keyword to indicate streaming semantics for the source: @@ -358,7 +403,7 @@ CREATE STREAMING TABLE basic_st AS SELECT * FROM STREAM samples.nyctaxi.trips; ``` -### Querying Tables Defined in Your Pipeline +### Querying Tables Defined in a Pipeline in SQL You can reference other tables defined in your pipeline: @@ -385,7 +430,7 @@ FROM customer_orders GROUP BY state, order_date; ``` -### Using Multiple Flows to Write to a Single Target +### Using Multiple Flows to Write to a Single Target in SQL You can create multiple flows that append data to the same target: @@ -393,13 +438,13 @@ You can create multiple flows that append data to the same target: -- create a streaming table CREATE STREAMING TABLE customers_us; --- add the first append flow -CREATE FLOW append1 +-- define the first append flow +CREATE FLOW append_customers_us_west AS INSERT INTO customers_us SELECT * FROM STREAM(customers_us_west); --- add the second append flow -CREATE FLOW append2 +-- define the second append flow +CREATE FLOW append_customers_us_east AS INSERT INTO customers_us SELECT * FROM STREAM(customers_us_east); ``` @@ -409,12 +454,15 @@ SELECT * FROM STREAM(customers_us_east); ### Python Considerations - SDP evaluates the code that defines a pipeline multiple times during planning and pipeline runs. Python functions that define datasets should include only the code required to define the table or view. -- The function used to define a dataset must return a Spark DataFrame. +- The function used to define a dataset must return a `pyspark.sql.DataFrame`. - Never use methods that save or write to files or tables as part of your SDP dataset code. +- When using the `for` loop pattern to define datasets in Python, ensure that the list of values passed to the `for` loop is always additive. + +Examples of Spark SQL operations that should never be used in SDP code: -Examples of Apache Spark operations that should never be used in SDP code: - `collect()` - `count()` +- `pivot()` - `toPandas()` - `save()` - `saveAsTable()` @@ -424,4 +472,3 @@ Examples of Apache Spark operations that should never be used in SDP code: ### SQL Considerations - The `PIVOT` clause is not supported in SDP SQL. -- When using the `for` loop pattern to define datasets in Python, ensure that the list of values passed to the `for` loop is always additive. From fceed47f9b08a110d6fd58c9d3fbce3e7584eac5 Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Mon, 12 Jan 2026 09:54:49 +0800 Subject: [PATCH 350/400] [SPARK-55005][SQL] Fix CONTINUE HANDLER to continue loop execution after handling exceptions in loop body MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR fixes a critical bug in `CONTINUE HANDLER` execution within loops. When a `CONTINUE HANDLER` handled an exception that occurred inside a loop body, the loop would exit completely instead of continuing to the next iteration. #### Root cause The `interruptConditionalStatements` method in `SqlScriptingExecution.scala` was designed to skip conditional statements when exceptions occurred in their condition evaluation (e.g., WHILE 1/0 > 0). However, it didn't distinguish between: - Exception in condition → loop should be skipped - Exception in body → loop should continue to next iteration The method unconditionally set `interrupted = true` on all conditional statements, causing loops to exit even when the error occurred during body execution. #### The fix The fix was to add `isInCondition: Boolean` method to `ConditionalStatementExec` trait to track whether a conditional statement is currently evaluating its condition or executing its body. It was also needed to implement `isInCondition` for all 6 `ConditionalStatementExec`: - IfElseStatementExec - WhileStatementExec - RepeatStatementExec - ForStatementExec - SearchedCaseStatementExec - SimpleCaseStatementExec ### Why are the changes needed? This ensures `CONTINUE HANDLER` only interrupts conditional statements when exceptions occur during condition evaluation, allowing loops to continue normally when exceptions occur in their bodies. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added comprehensive test coverage for `CONTINUE HANDLER` across all conditional statement types. These tests verify that when an exception occurs inside a loop body and is handled by a `CONTINUE HANDLER`, the loop continues to the next iteration rather than exiting. ### Was this patch authored or co-authored using generative AI tooling? Closes #53759 from miland-db/milan-dankovic_data/continue-handler-fix-loops. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan (cherry picked from commit 7a40891ac13413960b9c09ee04cdc63ada9195ac) Signed-off-by: Wenchen Fan --- .../sql/scripting/SqlScriptingExecution.scala | 7 +- .../scripting/SqlScriptingExecutionNode.scala | 28 ++ .../SqlScriptingExecutionSuite.scala | 357 ++++++++++++++++++ 3 files changed, 391 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala index 2a849aa2d6040..c8f7172e59bd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala @@ -102,7 +102,12 @@ class SqlScriptingExecution( currExecPlan match { case exec: ConditionalStatementExec => - exec.interrupted = true + // Only interrupt if the conditional statement is currently evaluating its condition. + // For loop statements, this means we should skip the loop when an exception occurs + // during condition evaluation, but NOT when an exception occurs in the loop body. + if (exec.isInCondition) { + exec.interrupted = true + } case _ => } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index aa2c2f405021a..c47df4b7a89ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -120,6 +120,22 @@ trait ConditionalStatementExec extends NonLeafStatementExec { * HANDLER. */ protected[scripting] var interrupted: Boolean = false + + /** + * Returns true if the conditional statement is currently evaluating its condition, + * false if it's executing its body. This is used by CONTINUE HANDLER to determine + * whether to interrupt the conditional statement when an exception occurs. + * + * For loop statements (WHILE, REPEAT, FOR), this should return true when evaluating + * the loop condition and false when executing the loop body. This distinction is + * critical because: + * - Exception in condition: loop should be skipped (interrupted) + * - Exception in body: loop should continue to next iteration (not interrupted) + * + * For IF/CASE statements, this should return true when evaluating the condition + * expression and false when executing any branch body. + */ + protected[scripting] def isInCondition: Boolean } /** @@ -479,6 +495,8 @@ class IfElseStatementExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + override protected[scripting] def isInCondition: Boolean = state == IfElseState.Condition + override def reset(): Unit = { state = IfElseState.Condition curr = Some(conditions.head) @@ -565,6 +583,8 @@ class WhileStatementExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + override protected[scripting] def isInCondition: Boolean = state == WhileState.Condition + override def reset(): Unit = { state = WhileState.Condition curr = Some(condition) @@ -654,6 +674,8 @@ class SearchedCaseStatementExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + override protected[scripting] def isInCondition: Boolean = state == CaseState.Condition + override def reset(): Unit = { state = CaseState.Condition curr = Some(conditions.head) @@ -793,6 +815,8 @@ class SimpleCaseStatementExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + override protected[scripting] def isInCondition: Boolean = state == CaseState.Condition + override def reset(): Unit = { state = CaseState.Condition bodyExec = None @@ -880,6 +904,8 @@ class RepeatStatementExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + override protected[scripting] def isInCondition: Boolean = state == RepeatState.Condition + override def reset(): Unit = { state = RepeatState.Body curr = Some(body) @@ -1215,6 +1241,8 @@ class ForStatementExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + override protected[scripting] def isInCondition: Boolean = state == ForState.VariableAssignment + override def reset(): Unit = { state = ForState.VariableAssignment isResultCacheValid = false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala index d080e1f05014d..2ad715f671edc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala @@ -1324,6 +1324,363 @@ class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(sqlScript, expected = expected) } + test("continue handler - should continue loop after handling error inside WHILE body") { + val sqlScript = + """ + |BEGIN + | DECLARE x, y = 1; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET y = -1; + | END; + | + | WHILE x < 5 DO + | SET x = x + 1; + | SET y = y / (x - 3); + | SELECT x, y; + | END WHILE; + | + | SELECT x, y; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(2, -1)), + Seq(Row(3, -1)), + Seq(Row(4, -1)), + Seq(Row(5, 0)), + Seq(Row(5, 0)) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue loop after handling error inside REPEAT body") { + val sqlScript = + """ + |BEGIN + | DECLARE x, y = 1; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET y = -1; + | END; + | + | REPEAT + | SET x = x + 1; + | SET y = y / (x - 3); + | SELECT x, y; + | UNTIL + | x >= 5 + | END REPEAT; + | + | SELECT x AS final_x, y AS final_y; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(2, -1)), // iteration 1: y = 1 / -1 = -1 + Seq(Row(3, -1)), // iteration 2: y = -1 / 0 handler sets y = -1 + Seq(Row(4, -1)), // iteration 3: y = -1 / 1 = -1 + Seq(Row(5, 0)), // iteration 4: y = -1 / 2 = 0 + Seq(Row(5, 0)) // final select + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue loop after handling error inside FOR body") { + val sqlScript = + """ + |BEGIN + | DECLARE y = 1; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET y = -1; + | END; + | + | FOR iter AS SELECT 1 AS val UNION ALL SELECT 2 UNION ALL SELECT 3 UNION ALL SELECT 4 + | UNION ALL SELECT 5 DO + | SET y = y / (iter.val - 3); + | SELECT iter.val, y; + | END FOR; + | + | SELECT y AS final_y; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1, 0)), // iteration 1: y = 1 / -2 = 0 + Seq(Row(2, 0)), // iteration 2: y = 0 / -1 = 0 + Seq(Row(3, -1)), // iteration 3: y = 0 / 0 handler sets y = -1 + Seq(Row(4, -1)), // iteration 4: y = -1 / 1 = -1 + Seq(Row(5, 0)), // iteration 5: y = -1 / 2 = 0 + Seq(Row(0)) // final select + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue after handling error inside IF body") { + val sqlScript = + """ + |BEGIN + | DECLARE x = 1; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = 1; + | END; + | + | IF x = 1 THEN + | SELECT 1 / 0; -- This will throw divide by zero + | SELECT 999; -- This should execute after handler + | END IF; + | + | SELECT handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(999)), // SELECT 999 executed after handler + Seq(Row(1)) // handled = 1 + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue after handling error inside ELSEIF body") { + val sqlScript = + """ + |BEGIN + | DECLARE x = 2; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = 1; + | END; + | + | IF x = 1 THEN + | SELECT 100; + | ELSEIF x = 2 THEN + | SELECT 1 / 0; -- This will throw divide by zero + | SELECT 888; -- This should execute after handler + | ELSE + | SELECT 200; + | END IF; + | + | SELECT handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(888)), // SELECT 888 executed after handler + Seq(Row(1)) // handled = 1 + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue after handling error inside ELSE body") { + val sqlScript = + """ + |BEGIN + | DECLARE x = 3; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = 1; + | END; + | + | IF x = 1 THEN + | SELECT 100; + | ELSEIF x = 2 THEN + | SELECT 200; + | ELSE + | SELECT 1 / 0; -- This will throw divide by zero + | SELECT 777; -- This should execute after handler + | END IF; + | + | SELECT handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(777)), // SELECT 777 executed after handler + Seq(Row(1)) // handled = 1 + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue after handling error inside SEARCHED CASE body") { + val sqlScript = + """ + |BEGIN + | DECLARE x = 2; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = 1; + | END; + | + | CASE + | WHEN x = 1 THEN SELECT 100; + | WHEN x = 2 THEN + | SELECT 1 / 0; -- This will throw divide by zero + | SELECT 666; -- This should execute after handler + | ELSE SELECT 200; + | END CASE; + | + | SELECT handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(666)), // SELECT 666 executed after handler + Seq(Row(1)) // handled = 1 + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - should continue after handling error inside SIMPLE CASE body") { + val sqlScript = + """ + |BEGIN + | DECLARE x = 2; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = 1; + | END; + | + | CASE x + | WHEN 1 THEN SELECT 100; + | WHEN 2 THEN + | SELECT 1 / 0; -- This will throw divide by zero + | SELECT 555; -- This should execute after handler + | ELSE SELECT 200; + | END CASE; + | + | SELECT handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(555)), // SELECT 555 executed after handler + Seq(Row(1)) // handled = 1 + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - nested WHILE loops with error in inner loop") { + val sqlScript = + """ + |BEGIN + | DECLARE outer = 0; + | DECLARE inner = 0; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = handled + 1; + | END; + | + | WHILE outer < 2 DO + | SET outer = outer + 1; + | SET inner = 0; + | WHILE inner < 3 DO + | SET inner = inner + 1; + | IF outer = 1 AND inner = 2 THEN + | SELECT 1 / 0; -- Error in inner loop, iteration 2 + | END IF; + | SELECT outer, inner, handled; + | END WHILE; + | END WHILE; + | + | SELECT outer AS final_outer, handled AS final_handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1, 1, 0)), // outer=1, inner=1, no error yet + Seq(Row(1, 2, 1)), // outer=1, inner=2, error handled + Seq(Row(1, 3, 1)), // outer=1, inner=3, continuing + Seq(Row(2, 1, 1)), // outer=2, inner=1 + Seq(Row(2, 2, 1)), // outer=2, inner=2, no error this time + Seq(Row(2, 3, 1)), // outer=2, inner=3 + Seq(Row(2, 1)) // final select + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - nested REPEAT loops with error in inner loop") { + val sqlScript = + """ + |BEGIN + | DECLARE outer = 0; + | DECLARE inner = 0; + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = handled + 1; + | END; + | + | REPEAT + | SET outer = outer + 1; + | SET inner = 0; + | REPEAT + | SET inner = inner + 1; + | IF outer = 2 AND inner = 1 THEN + | SELECT 1 / 0; -- Error in inner loop + | END IF; + | SELECT outer, inner, handled; + | UNTIL inner >= 2 + | END REPEAT; + | UNTIL outer >= 2 + | END REPEAT; + | + | SELECT outer AS final_outer, handled AS final_handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1, 1, 0)), // outer=1, inner=1 + Seq(Row(1, 2, 0)), // outer=1, inner=2 + Seq(Row(2, 1, 1)), // outer=2, inner=1, error handled + Seq(Row(2, 2, 1)), // outer=2, inner=2 + Seq(Row(2, 1)) // final select + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - nested FOR loops with error in inner loop") { + val sqlScript = + """ + |BEGIN + | DECLARE handled = 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SET handled = handled + 1; + | END; + | + | FOR o AS SELECT 1 AS val UNION ALL SELECT 2 DO + | FOR i AS SELECT 1 AS val UNION ALL SELECT 2 UNION ALL SELECT 3 DO + | IF o.val = 1 AND i.val = 3 THEN + | SELECT 1 / 0; -- Error in inner loop + | END IF; + | SELECT o.val AS outer, i.val AS inner, handled; + | END FOR; + | END FOR; + | + | SELECT handled AS final_handled; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1, 1, 0)), // o=1, i=1 + Seq(Row(1, 2, 0)), // o=1, i=2 + Seq(Row(1, 3, 1)), // o=1, i=3, error handled + Seq(Row(2, 1, 1)), // o=2, i=1 + Seq(Row(2, 2, 1)), // o=2, i=2 + Seq(Row(2, 3, 1)), // o=2, i=3 + Seq(Row(1)) // final select + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + test("exit handler body without BEGIN-END propagates error properly") { val sqlScript = """ From 4fa0bdc1efa80b15617b7b00b6ced6975416fe1f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 12 Jan 2026 14:44:58 +0900 Subject: [PATCH 351/400] [SPARK-54979][BUILD] Upgrade ORC to 2.2.2 ### What changes were proposed in this pull request? This PR aims to upgrade ORC to 2.2.2. ### Why are the changes needed? To use the latest bug fixed version. - https://github.com/apache/orc/releases/tag/v2.2.2 - https://orc.apache.org/news/2026/01/11/ORC-2.2.2/ - https://repo1.maven.org/maven2/org/apache/orc/orc-core/2.2.2/ ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53743 from dongjoon-hyun/SPARK-54979. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 5fed54b074b3192ce7b843a95fa89795ddde7a41) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 6 +++--- pom.xml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index ef5d6bae09f51..327e0a8b0b6a8 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -243,10 +243,10 @@ opencsv/2.3//opencsv-2.3.jar opentracing-api/0.33.0//opentracing-api-0.33.0.jar opentracing-noop/0.33.0//opentracing-noop-0.33.0.jar opentracing-util/0.33.0//opentracing-util-0.33.0.jar -orc-core/2.2.1/shaded-protobuf/orc-core-2.2.1-shaded-protobuf.jar +orc-core/2.2.2/shaded-protobuf/orc-core-2.2.2-shaded-protobuf.jar orc-format/1.1.1/shaded-protobuf/orc-format-1.1.1-shaded-protobuf.jar -orc-mapreduce/2.2.1/shaded-protobuf/orc-mapreduce-2.2.1-shaded-protobuf.jar -orc-shims/2.2.1//orc-shims-2.2.1.jar +orc-mapreduce/2.2.2/shaded-protobuf/orc-mapreduce-2.2.2-shaded-protobuf.jar +orc-shims/2.2.2//orc-shims-2.2.2.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8.3//paranamer-2.8.3.jar diff --git a/pom.xml b/pom.xml index f6eabc8c37c6c..1d76def7d2fe8 100644 --- a/pom.xml +++ b/pom.xml @@ -142,7 +142,7 @@ 10.16.1.1 1.16.0 - 2.2.1 + 2.2.2 shaded-protobuf 11.0.26 5.0.0 From a39c1b8e5e2506d0da530f98dcd1c55558f98282 Mon Sep 17 00:00:00 2001 From: Chris Boumalhab Date: Mon, 12 Jan 2026 13:52:23 -0800 Subject: [PATCH 352/400] [SPARK-54785][SQL] Add support for binary sketch aggregations in KLL This PR adds SQL aggregate functions with their tests for the KLL merge aggregate functions: - `kll_merge_agg_bigint` - `kll_merge_agg_float` - `kll_merge_agg_double` These aggregate functions merge multiple binary KLL sketch representations. Initial PRs: - https://github.com/apache/spark/pull/52900/ - https://github.com/apache/spark/pull/52800/ The existing scalar `kll_sketch_merge_*` functions can only merge two sketches at a time. In distributed computing scenarios where sketches are pre-computed across multiple partitions, time windows, or datasets, users need to merge many sketches together. Yes, this PR adds 3 new aggregate functions. New SQL tests were added to `sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql`: **Positive tests:** - Merging bigint/float/double sketches from multiple rows - Merging with custom k parameters (400, 300, 500) - NULL value handling **Negative tests:** - Type mismatches (passing non-binary types) - Invalid binary data - k parameter validation (too small, too large, NULL, non-constant) claude-4.5-sonnet and manual changes. Closes #53548 from cboumalh/cboumalh-kll-enhancement. Lead-authored-by: Chris Boumalhab Co-authored-by: Chris Boumalhab <84485659+cboumalh@users.noreply.github.com> Signed-off-by: Daniel Tenedorio (cherry picked from commit fc15f726eabd0a069dc47b195e1a91aa6b1bf541) Signed-off-by: Daniel Tenedorio --- docs/sql-ref-sketch-aggregates.md | 988 ++++++++++++++++++ .../reference/pyspark.sql/functions.rst | 3 + .../pyspark/sql/connect/functions/builtin.py | 42 + python/pyspark/sql/functions/__init__.py | 3 + python/pyspark/sql/functions/builtin.py | 132 +++ python/pyspark/sql/tests/test_functions.py | 91 ++ .../org/apache/spark/sql/functions.scala | 296 ++++-- .../catalyst/analysis/FunctionRegistry.scala | 33 +- .../expressions/aggregate/kllAggregates.scala | 440 +++++++- .../sql-functions/sql-expression-schema.md | 3 + .../analyzer-results/kllquantiles.sql.out | 485 +++++++++ .../sql-tests/inputs/kllquantiles.sql | 206 +++- .../sql-tests/results/kllquantiles.sql.out | 454 ++++++++ .../spark/sql/DataFrameAggregateSuite.scala | 104 ++ 14 files changed, 3150 insertions(+), 130 deletions(-) create mode 100644 docs/sql-ref-sketch-aggregates.md diff --git a/docs/sql-ref-sketch-aggregates.md b/docs/sql-ref-sketch-aggregates.md new file mode 100644 index 0000000000000..6b92ba7b3c9e4 --- /dev/null +++ b/docs/sql-ref-sketch-aggregates.md @@ -0,0 +1,988 @@ +--- +layout: global +title: Sketch-Based Approximate Functions +displayTitle: Sketch-Based Approximate Functions +license: | + 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. +--- + +Spark's SQL and DataFrame APIs provide a collection of sketch-based approximate functions powered by the [Apache DataSketches](https://datasketches.apache.org/) library. These functions enable efficient probabilistic computations on large datasets with bounded memory usage and accuracy guarantees. + +Sketches are compact data structures that summarize large datasets, supporting distributed aggregation through serialization and merging. This makes them ideal for use cases including (so far): +- **Approximate count distinct** (HLL and Theta sketches) +- **Approximate quantile estimation** (KLL sketches) +- **Approximate frequent items** (Top-K sketches) +- **Set operations** on distinct counts (Theta sketches) + +### Table of Contents + +* [HyperLogLog (HLL) Sketch Functions](#hyperloglog-hll-sketch-functions) + * [hll_sketch_agg](#hll_sketch_agg) + * [hll_union_agg](#hll_union_agg) + * [hll_sketch_estimate](#hll_sketch_estimate) + * [hll_union](#hll_union) +* [Theta Sketch Functions](#theta-sketch-functions) + * [theta_sketch_agg](#theta_sketch_agg) + * [theta_union_agg](#theta_union_agg) + * [theta_intersection_agg](#theta_intersection_agg) + * [theta_sketch_estimate](#theta_sketch_estimate) + * [theta_union](#theta_union) + * [theta_intersection](#theta_intersection) + * [theta_difference](#theta_difference) +* [KLL Quantile Sketch Functions](#kll-quantile-sketch-functions) + * [kll_sketch_agg_*](#kll_sketch_agg_) + * [kll_merge_agg_*](#kll_merge_agg_) + * [kll_sketch_to_string_*](#kll_sketch_to_string_) + * [kll_sketch_get_n_*](#kll_sketch_get_n_) + * [kll_sketch_merge_*](#kll_sketch_merge_) + * [kll_sketch_get_quantile_*](#kll_sketch_get_quantile_) + * [kll_sketch_get_rank_*](#kll_sketch_get_rank_) +* [Approximate Top-K Functions](#approximate-top-k-functions) + * [approx_top_k_accumulate](#approx_top_k_accumulate) + * [approx_top_k_combine](#approx_top_k_combine) + * [approx_top_k_estimate](#approx_top_k_estimate) +* [Best Practices](#best-practices) + * [Choosing Between HLL and Theta Sketches](#choosing-between-hll-and-theta-sketches) + * [Accuracy vs. Memory Trade-offs](#accuracy-vs-memory-trade-offs) + * [Storing and Reusing Sketches](#storing-and-reusing-sketches) +* [Common Use Cases and Examples](#common-use-cases-and-examples) + * [Example: Tracking Daily Unique Users with HLL Sketches](#example-tracking-daily-unique-users-with-hll-sketches) + * [Example: Computing Percentiles Over Time with KLL Sketches](#example-computing-percentiles-over-time-with-kll-sketches) + * [Example: Set Operations with Theta Sketches](#example-set-operations-with-theta-sketches) + * [Example: Finding Trending Items with Top-K Sketches](#example-finding-trending-items-with-top-k-sketches) + +--- + +## HyperLogLog (HLL) Sketch Functions + +HyperLogLog sketches provide approximate count distinct functionality with configurable accuracy and memory usage. They are well-suited for counting unique values in very large datasets. + +See the [Apache DataSketches HLL documentation](https://datasketches.apache.org/docs/HLL/HLL.html) for more information. + +### hll_sketch_agg + +Creates an HLL sketch from input values that can later be used to estimate count distinct. + +**Syntax:** +```sql +hll_sketch_agg(expr [, lgConfigK]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `expr` | INT, BIGINT, STRING, or BINARY | The expression whose distinct values will be counted | +| `lgConfigK` | INT (optional) | Log-base-2 of K, where K is the number of buckets. Range: 4-21. Default: 12. Higher values provide more accuracy but use more memory. | + +Returns a BINARY containing the HLL sketch in updatable binary representation. + +**Examples:** +```sql +-- Basic usage: create a sketch and estimate distinct count +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES (1), (1), (2), (2), (3) tab(col); +-- Result: 3 + +-- With custom lgConfigK for higher accuracy +SELECT hll_sketch_estimate(hll_sketch_agg(col, 16)) +FROM VALUES (50), (60), (60), (60), (75), (100) tab(col); +-- Result: 4 + +-- With string values +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES ('abc'), ('def'), ('abc'), ('ghi'), ('abc') tab(col); +-- Result: 3 +``` + +**Notes:** +- NULL values are ignored during aggregation. +- Empty strings (for STRING type) and empty byte arrays (for BINARY type) are ignored. +- The sketch can be stored and later merged with other sketches using `hll_union` or `hll_union_agg`. + +--- + +### hll_union_agg + +Aggregates multiple HLL sketches into a single merged sketch. + +**Syntax:** +```sql +hll_union_agg(sketch [, allowDifferentLgConfigK]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | An HLL sketch in binary format (produced by `hll_sketch_agg`) | +| `allowDifferentLgConfigK` | BOOLEAN (optional) | If true, allows merging sketches with different lgConfigK values. Default: false. | + +Returns a BINARY containing the merged HLL sketch. + +**Examples:** +```sql +-- Merge sketches from different partitions +SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) +FROM ( + SELECT hll_sketch_agg(col) as sketch + FROM VALUES (1) tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) tab(col) +); +-- Result: 1 + +-- Standard merge (same lgConfigK) +SELECT hll_sketch_estimate(hll_union_agg(sketch)) +FROM ( + SELECT hll_sketch_agg(col) as sketch + FROM VALUES (1), (2) tab(col) + UNION ALL + SELECT hll_sketch_agg(col) as sketch + FROM VALUES (3), (4) tab(col) +); +-- Result: 4 +``` + +**Notes:** +- If `allowDifferentLgConfigK` is false and sketches have different lgConfigK values, an error is thrown. +- The output sketch uses the minimum lgConfigK value of all input sketches when merging sketches with different sizes. + +--- + +### hll_sketch_estimate + +Estimates the number of unique values from an HLL sketch. + +**Syntax:** +```sql +hll_sketch_estimate(sketch) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | An HLL sketch in binary format | + +Returns a BIGINT representing the estimated count of distinct values. + +**Examples:** +```sql +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES (1), (1), (2), (2), (3) tab(col); +-- Result: 3 +``` + +**Errors:** +- Throws an error if the input is not a valid HLL sketch binary representation. + +--- + +### hll_union + +Merges two HLL sketches into one (scalar function). + +**Syntax:** +```sql +hll_union(first, second [, allowDifferentLgConfigK]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `first` | BINARY | First HLL sketch | +| `second` | BINARY | Second HLL sketch | +| `allowDifferentLgConfigK` | BOOLEAN (optional) | Allow different lgConfigK values. Default: false. | + +Returns a BINARY containing the merged HLL sketch. + +**Examples:** +```sql +SELECT hll_sketch_estimate( + hll_union( + hll_sketch_agg(col1), + hll_sketch_agg(col2))) +FROM VALUES (1, 4), (1, 4), (2, 5), (2, 5), (3, 6) tab(col1, col2); +-- Result: 6 +``` + +--- + +## Theta Sketch Functions + +Theta sketches provide approximate count distinct with support for set operations (union, intersection, and difference). This makes them ideal for computing unique counts across overlapping datasets. + +See the [Apache DataSketches Theta documentation](https://datasketches.apache.org/docs/Theta/ThetaSketches.html) for more information. + +### theta_sketch_agg + +Creates a Theta sketch from input values. + +**Syntax:** +```sql +theta_sketch_agg(expr [, lgNomEntries]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `expr` | INT, BIGINT, FLOAT, DOUBLE, STRING, BINARY, ARRAY<INT>, or ARRAY<BIGINT> | The expression whose distinct values will be counted | +| `lgNomEntries` | INT (optional) | Log-base-2 of nominal entries. Range: 4-26. Default: 12. | + +Returns a BINARY containing the Theta sketch in compact binary representation. + +**Examples:** +```sql +-- Basic distinct count +SELECT theta_sketch_estimate(theta_sketch_agg(col)) +FROM VALUES (1), (1), (2), (2), (3) tab(col); +-- Result: 3 + +-- With custom lgNomEntries +SELECT theta_sketch_estimate(theta_sketch_agg(col, 22)) +FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col); +-- Result: 7 + +-- With array values +SELECT theta_sketch_estimate(theta_sketch_agg(col)) +FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)), (ARRAY(1, 2)) tab(col); +-- Result: 2 +``` + +**Notes:** +- NULL values are ignored. +- Supports a wider range of input types compared to HLL sketches. +- Empty arrays, empty strings, and empty binary values are ignored. + +--- + +### theta_union_agg + +Aggregates multiple Theta sketches using union operation. + +**Syntax:** +```sql +theta_union_agg(sketch [, lgNomEntries]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A Theta sketch in binary format | +| `lgNomEntries` | INT (optional) | Log-base-2 of nominal entries. Range: 4-26. Default: 12. | + +Returns a BINARY containing the merged Theta sketch. + +**Examples:** +```sql +SELECT theta_sketch_estimate(theta_union_agg(sketch, 15)) +FROM ( + SELECT theta_sketch_agg(col1) as sketch + FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col1) + UNION ALL + SELECT theta_sketch_agg(col2, 20) as sketch + FROM VALUES (5), (6), (7), (8), (9), (10), (11) tab(col2) +); +-- Result: 11 +``` + +--- + +### theta_intersection_agg + +Aggregates multiple Theta sketches using intersection operation (finds common distinct values). + +**Syntax:** +```sql +theta_intersection_agg(sketch) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A Theta sketch in binary format | + +Returns a BINARY containing the intersected Theta sketch. + +**Examples:** +```sql +SELECT theta_sketch_estimate(theta_intersection_agg(sketch)) +FROM ( + SELECT theta_sketch_agg(col1) as sketch + FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col1) + UNION ALL + SELECT theta_sketch_agg(col2) as sketch + FROM VALUES (5), (6), (7), (8), (9), (10), (11) tab(col2) +); +-- Result: 3 (values 5, 6, 7 are common) +``` + +--- + +### theta_sketch_estimate + +Estimates the number of unique values from a Theta sketch. + +**Syntax:** +```sql +theta_sketch_estimate(sketch) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A Theta sketch in binary format | + +Returns a BIGINT representing the estimated count of distinct values. + +**Examples:** +```sql +SELECT theta_sketch_estimate(theta_sketch_agg(col)) +FROM VALUES (1), (1), (2), (2), (3) tab(col); +-- Result: 3 +``` + +--- + +### theta_union + +Merges two Theta sketches using union (scalar function). + +**Syntax:** +```sql +theta_union(first, second [, lgNomEntries]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `first` | BINARY | First Theta sketch | +| `second` | BINARY | Second Theta sketch | +| `lgNomEntries` | INT (optional) | Log-base-2 of nominal entries. Range: 4-26. Default: 12. | + +Returns a BINARY containing the merged Theta sketch. + +**Examples:** +```sql +SELECT theta_sketch_estimate( + theta_union( + theta_sketch_agg(col1), + theta_sketch_agg(col2))) +FROM VALUES (1, 4), (1, 4), (2, 5), (2, 5), (3, 6) tab(col1, col2); +-- Result: 6 +``` + +--- + +### theta_intersection + +Computes the intersection of two Theta sketches (scalar function). + +**Syntax:** +```sql +theta_intersection(first, second) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `first` | BINARY | First Theta sketch | +| `second` | BINARY | Second Theta sketch | + +Returns a BINARY containing the intersected Theta sketch. + +**Examples:** +```sql +SELECT theta_sketch_estimate( + theta_intersection( + theta_sketch_agg(col1), + theta_sketch_agg(col2))) +FROM VALUES (5, 4), (1, 4), (2, 5), (2, 5), (3, 1) tab(col1, col2); +-- Result: 2 (values 1 and 5 are common) +``` + +--- + +### theta_difference + +Computes the set difference of two Theta sketches (A - B). + +**Syntax:** +```sql +theta_difference(first, second) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `first` | BINARY | First Theta sketch (A) | +| `second` | BINARY | Second Theta sketch (B) | + +Returns a BINARY containing a Theta sketch representing values in A but not in B. + +**Examples:** +```sql +SELECT theta_sketch_estimate( + theta_difference( + theta_sketch_agg(col1), + theta_sketch_agg(col2))) +FROM VALUES (5, 4), (1, 4), (2, 5), (2, 5), (3, 1) tab(col1, col2); +-- Result: 2 (values 2 and 3 are in col1 but not col2) +``` + +--- + +## KLL Quantile Sketch Functions + +KLL (K-Linear-Logarithmic) sketches provide approximate quantile estimation. They are useful for computing percentiles, medians, and other order statistics on large datasets without sorting. + +See the [Apache DataSketches KLL documentation](https://datasketches.apache.org/docs/KLL/KLLSketch.html) for more information. + +KLL functions are type-specific to avoid precision loss: +- **BIGINT** variants: For integer types (TINYINT, SMALLINT, INT, BIGINT) +- **FLOAT** variants: For FLOAT values only +- **DOUBLE** variants: For FLOAT and DOUBLE values + +### kll_sketch_agg_* + +Creates a KLL sketch from numeric values for quantile estimation. + +**Syntax:** +```sql +kll_sketch_agg_bigint(expr [, k]) +kll_sketch_agg_float(expr [, k]) +kll_sketch_agg_double(expr [, k]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `expr` | Numeric (see variants above) | The numeric column to summarize | +| `k` | INT (optional) | Controls accuracy and size. Range: 8-65535. Default: 200 (~1.65% normalized rank error). | + +Returns a BINARY containing the KLL sketch in compact binary representation. + +**Examples:** +```sql +-- Get median (0.5 quantile) +SELECT kll_sketch_get_quantile_bigint(kll_sketch_agg_bigint(col), 0.5) +FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col); +-- Result: 4 + +-- With custom k for higher accuracy +SELECT kll_sketch_get_quantile_bigint(kll_sketch_agg_bigint(col, 400), 0.5) +FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col); +-- Result: 4 +``` + +**Notes:** +- Use the appropriate variant to avoid precision loss: use `_bigint` for integers, `_float` for floats, `_double` for doubles. +- NULL values are ignored during aggregation. + +--- + +### kll_merge_agg_* + +Aggregates multiple KLL sketches of the same type by merging them together. This is useful for combining sketches created in separate aggregations (e.g., from different partitions or time windows). These are aggregate functions. + +**Syntax:** +```sql +kll_merge_agg_bigint(sketch [, k]) +kll_merge_agg_float(sketch [, k]) +kll_merge_agg_double(sketch [, k]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A KLL sketch in binary format (e.g., from `kll_sketch_agg_*`) | +| `k` | INT (optional) | Controls accuracy and size of the merged sketch. Range: 8-65535. If not specified, the merged sketch adopts the k value from the first input sketch. | + +Returns a BINARY containing the merged KLL sketch. + +**Examples:** +```sql +-- Merge sketches from different partitions +SELECT kll_sketch_get_quantile_bigint( + kll_merge_agg_bigint(sketch), + 0.5 +) +FROM ( + SELECT kll_sketch_agg_bigint(col) as sketch + FROM VALUES (1), (2), (3) tab(col) + UNION ALL + SELECT kll_sketch_agg_bigint(col) as sketch + FROM VALUES (4), (5), (6) tab(col) +); +-- Result: 3 + +-- Get the total count from merged sketches +SELECT kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch)) +FROM ( + SELECT kll_sketch_agg_bigint(col) as sketch + FROM VALUES (1), (2), (3) tab(col) + UNION ALL + SELECT kll_sketch_agg_bigint(col) as sketch + FROM VALUES (4), (5), (6) tab(col) +); +-- Result: 6 +``` + +**Notes:** +- When `k` is not specified, the merged sketch adopts the k value from the first input sketch. +- The merge operation can handle input sketches with different k values. +- NULL values are ignored during aggregation. +- Use this function when you need to merge multiple sketches in an aggregation context. For merging exactly two sketches, use the scalar `kll_sketch_merge_*` functions instead. + +--- + +### kll_sketch_to_string_* + +Returns a human-readable summary of the sketch. + +**Syntax:** +```sql +kll_sketch_to_string_bigint(sketch) +kll_sketch_to_string_float(sketch) +kll_sketch_to_string_double(sketch) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A KLL sketch of the corresponding type | + +Returns a STRING containing a human-readable summary including sketch parameters and statistics. + +--- + +### kll_sketch_get_n_* + +Returns the number of items collected in the sketch. + +**Syntax:** +```sql +kll_sketch_get_n_bigint(sketch) +kll_sketch_get_n_float(sketch) +kll_sketch_get_n_double(sketch) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A KLL sketch of the corresponding type | + +Returns a BIGINT representing the count of items in the sketch. + +**Examples:** +```sql +SELECT kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col)) +FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col); +-- Result: 7 +``` + +--- + +### kll_sketch_merge_* + +Merges two KLL sketches of the same type. These are scalar functions. + +**Syntax:** +```sql +kll_sketch_merge_bigint(left, right) +kll_sketch_merge_float(left, right) +kll_sketch_merge_double(left, right) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `left` | BINARY | First KLL sketch | +| `right` | BINARY | Second KLL sketch (must be same type as left) | + +Returns a BINARY containing the merged KLL sketch. + +**Examples:** +```sql +-- Merge two sketches from different data partitions +SELECT kll_sketch_get_quantile_bigint( + kll_sketch_merge_bigint( + kll_sketch_agg_bigint(col1), + kll_sketch_agg_bigint(col2)), 0.5) +FROM VALUES (1, 6), (2, 7), (3, 8), (4, 9), (5, 10) tab(col1, col2); +-- Result: approximately 5 (median of 1-10) +``` + +**Errors:** +- Throws an error if sketches are of incompatible types or formats. + +**Notes:** +- The merge operation can handle input sketches with different k values. +- Use this function when you need to merge exactly two sketches in an scalar context. For merging multiple sketches in an aggregation context, use the aggregate `kll_merge_agg_*` functions instead. + +--- + +### kll_sketch_get_quantile_* + +Gets the approximate value at a given quantile rank. + +**Syntax:** +```sql +kll_sketch_get_quantile_bigint(sketch, rank) +kll_sketch_get_quantile_float(sketch, rank) +kll_sketch_get_quantile_double(sketch, rank) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A KLL sketch of the corresponding type | +| `rank` | DOUBLE or ARRAY<DOUBLE> | Quantile rank(s) between 0.0 and 1.0. Use 0.5 for median, 0.95 for 95th percentile, etc. | + +Returns the approximate value at the given quantile: +- If `rank` is a scalar: Returns the corresponding type (BIGINT, FLOAT, or DOUBLE) +- If `rank` is an array: Returns ARRAY of the corresponding type + +**Examples:** +```sql +-- Get the median +SELECT kll_sketch_get_quantile_bigint(kll_sketch_agg_bigint(col), 0.5) +FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col); +-- Result: 4 + +-- Get multiple percentiles at once +SELECT kll_sketch_get_quantile_bigint( + kll_sketch_agg_bigint(col), + ARRAY(0.25, 0.5, 0.75, 0.95)) +FROM VALUES (1), (2), (3), (4), (5), (6), (7), (8), (9), (10) tab(col); +-- Result: Array of values at 25th, 50th, 75th, and 95th percentiles +``` + +**Errors:** +- Throws an error if rank values are outside [0.0, 1.0]. +- Returns NULL if the input sketch is NULL. + +--- + +### kll_sketch_get_rank_* + +Gets the normalized rank (0.0 to 1.0) of a given value in the sketch's distribution. + +**Syntax:** +```sql +kll_sketch_get_rank_bigint(sketch, value) +kll_sketch_get_rank_float(sketch, value) +kll_sketch_get_rank_double(sketch, value) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `sketch` | BINARY | A KLL sketch of the corresponding type | +| `value` | Corresponding type (BIGINT, FLOAT, or DOUBLE) | The value to find the rank for | + +Returns a DOUBLE representing the normalized rank between 0.0 and 1.0. + +**Examples:** +```sql +-- Find what percentile the value 3 is at +SELECT kll_sketch_get_rank_bigint(kll_sketch_agg_bigint(col), 3) +FROM VALUES (1), (2), (3), (4), (5), (6), (7) tab(col); +-- Result: approximately 0.43 (3 is around the 43rd percentile) +``` + +--- + +## Approximate Top-K Functions + +Top-K functions estimate the most frequent items (heavy hitters) in a dataset using the DataSketches Frequent Items sketch. + +See the [Apache DataSketches Frequency documentation](https://datasketches.apache.org/docs/Frequency/FrequencySketches.html) for more information. + +### approx_top_k_accumulate + +Creates a sketch that can be stored and later combined or estimated. Useful for pre-aggregating data. + +**Syntax:** +```sql +approx_top_k_accumulate(expr [, maxItemsTracked]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `expr` | Same as `approx_top_k` | The column to accumulate | +| `maxItemsTracked` | INT (optional) | Maximum items tracked. Range: 1 to 1,000,000. Default: 10,000. | + +Returns a STRUCT containing a sketch state that can be passed to `approx_top_k_combine` or `approx_top_k_estimate`. + +**Examples:** +```sql +-- Accumulate then estimate +SELECT approx_top_k_estimate(approx_top_k_accumulate(expr)) +FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) tab(expr); +-- Result: [{"item":0,"count":2},{"item":4,"count":2},{"item":1,"count":2},{"item":2,"count":1},{"item":3,"count":1}] +``` + +--- + +### approx_top_k_combine + +Combines multiple sketches into a single sketch. + +**Syntax:** +```sql +approx_top_k_combine(state [, maxItemsTracked]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `state` | STRUCT | A sketch state from `approx_top_k_accumulate` or `approx_top_k_combine` | +| `maxItemsTracked` | INT (optional) | If specified, sets the combined sketch size. If not specified, all input sketches must have the same maxItemsTracked. | + +Returns a STRUCT containing the combined sketch state. + +**Examples:** +```sql +-- Combine sketches from different partitions +SELECT approx_top_k_estimate(approx_top_k_combine(sketch, 10000), 5) +FROM ( + SELECT approx_top_k_accumulate(expr) AS sketch + FROM VALUES (0), (0), (1), (1) tab(expr) + UNION ALL + SELECT approx_top_k_accumulate(expr) AS sketch + FROM VALUES (2), (3), (4), (4) tab(expr) +); +-- Result: [{"item":0,"count":2},{"item":4,"count":2},{"item":1,"count":2},{"item":2,"count":1},{"item":3,"count":1}] +``` + +**Errors:** +- Throws an error if input sketches have different `maxItemsTracked` values and no explicit value is provided. +- Throws an error if input sketches have different item data types. + +--- + +### approx_top_k_estimate + +Extracts the top K items from a sketch. + +**Syntax:** +```sql +approx_top_k_estimate(state [, k]) +``` + +| Argument | Type | Description | +|----------|------|-------------| +| `state` | STRUCT | A sketch state from `approx_top_k_accumulate` or `approx_top_k_combine` | +| `k` | INT (optional) | Number of top items to return. Default: 5. | + +Returns an ARRAY<STRUCT<item, count>> containing the frequent items sorted by count descending. + +**Examples:** +```sql +SELECT approx_top_k_estimate(approx_top_k_accumulate(expr), 2) +FROM VALUES 'a', 'b', 'c', 'c', 'c', 'c', 'd', 'd' tab(expr); +-- Result: [{"item":"c","count":4},{"item":"d","count":2}] +``` + +--- + +## Best Practices + +### Choosing Between HLL and Theta Sketches + +| Use Case | Recommended Sketch | +|----------|-------------------| +| Simple count distinct | HLL (more memory efficient) | +| Set operations (union, intersection, difference) | Theta | +| Very high cardinality with moderate accuracy | HLL with higher lgConfigK | +| Need to compute overlaps between datasets | Theta | + +### Accuracy vs. Memory Trade-offs + +| Sketch Type | Parameter | Effect of Increasing | +|-------------|-----------|---------------------| +| HLL | lgConfigK | Higher accuracy, more memory (2^lgConfigK bytes) | +| Theta | lgNomEntries | Higher accuracy, more memory (8 * 2^lgNomEntries bytes) | +| KLL | k | Higher accuracy, more memory | +| Top-K | maxItemsTracked | Better heavy-hitter detection, more memory | + +### Storing and Reusing Sketches + +Sketches can be stored in BINARY columns and later merged: + +```sql +-- Create a table to store daily sketches +CREATE TABLE daily_user_sketches ( + date DATE, + user_sketch BINARY +); + +-- Insert daily sketches +INSERT INTO daily_user_sketches +SELECT current_date(), hll_sketch_agg(user_id) +FROM events; + +-- Compute weekly unique users by merging daily sketches +SELECT hll_sketch_estimate(hll_union_agg(user_sketch)) +FROM daily_user_sketches +WHERE date BETWEEN '2024-01-01' AND '2024-01-07'; +``` + +--- + +## Common Use Cases and Examples + +Sketches are particularly valuable for periodic ETL jobs where you need to maintain running statistics across multiple batches of data. The general workflow is: + +1. **Aggregate** input values into a sketch using an aggregate function +2. **Store** the sketch (as BINARY) in a table +3. **Merge** new sketches with previously stored sketches +4. **Query** the final sketch to get approximate answers + +### Example: Tracking Daily Unique Users with HLL Sketches + +This example shows how to maintain a running count of unique users across daily batches. + +```sql +-- Create a table to store daily HLL sketches +CREATE TABLE daily_user_sketches ( + event_date DATE, + user_sketch BINARY +) USING PARQUET; + +-- Day 1: Process first batch of events and store the sketch +INSERT INTO daily_user_sketches +SELECT + DATE'2024-01-01' as event_date, + hll_sketch_agg(user_id) as user_sketch +FROM day1_events; + +-- Day 2: Process second batch and store its sketch +INSERT INTO daily_user_sketches +SELECT + DATE'2024-01-02' as event_date, + hll_sketch_agg(user_id) as user_sketch +FROM day2_events; + +-- Query: Get unique users for a single day +SELECT + event_date, + hll_sketch_estimate(user_sketch) as unique_users +FROM daily_user_sketches +WHERE event_date = DATE'2024-01-01'; + +-- Query: Get unique users across a date range (merging sketches) +SELECT hll_sketch_estimate(hll_union_agg(user_sketch)) as unique_users_in_week +FROM daily_user_sketches +WHERE event_date BETWEEN DATE'2024-01-01' AND DATE'2024-01-07'; +``` + +### Example: Computing Percentiles Over Time with KLL Sketches + +This example shows how to track response time percentiles across hourly batches. + +```sql +-- Create a table to store hourly KLL sketches for response times +CREATE TABLE hourly_latency_sketches ( + hour_ts TIMESTAMP, + latency_sketch BINARY +) USING PARQUET; + +-- Process each hour's data and store the sketch +INSERT INTO hourly_latency_sketches +SELECT + DATE_TRUNC('hour', event_time) as hour_ts, + kll_sketch_agg_bigint(response_time_ms) as latency_sketch +FROM hourly_events +GROUP BY DATE_TRUNC('hour', event_time); + +-- Query: Get p50, p95, p99 for a specific hour +SELECT + hour_ts, + kll_sketch_get_quantile_bigint(latency_sketch, 0.5) as p50_ms, + kll_sketch_get_quantile_bigint(latency_sketch, 0.95) as p95_ms, + kll_sketch_get_quantile_bigint(latency_sketch, 0.99) as p99_ms +FROM hourly_latency_sketches +WHERE hour_ts = TIMESTAMP'2024-01-15 14:00:00'; + +-- Query: Get percentiles across a full day by merging hourly sketches +WITH daily_sketch AS ( + SELECT kll_merge_agg_bigint(latency_sketch) as merged_sketch + FROM hourly_latency_sketches + WHERE DATE(hour_ts) = DATE'2024-01-15' +) +SELECT + kll_sketch_get_quantile_bigint(merged_sketch, 0.5) as p50_ms, + kll_sketch_get_quantile_bigint(merged_sketch, 0.95) as p95_ms, + kll_sketch_get_quantile_bigint(merged_sketch, 0.99) as p99_ms +FROM daily_sketch; +``` + +### Example: Set Operations with Theta Sketches + +Theta sketches support set operations, making them useful for analyzing overlapping populations. + +```sql +-- Create sketches for users who performed different actions +CREATE TABLE action_sketches ( + action_type STRING, + user_sketch BINARY +) USING PARQUET; + +-- Store sketches for each action type +INSERT INTO action_sketches +SELECT 'purchase', theta_sketch_agg(user_id) FROM purchases; + +INSERT INTO action_sketches +SELECT 'add_to_cart', theta_sketch_agg(user_id) FROM cart_additions; + +INSERT INTO action_sketches +SELECT 'page_view', theta_sketch_agg(user_id) FROM page_views; + +-- Query: How many users purchased? +SELECT theta_sketch_estimate(user_sketch) as purchasers +FROM action_sketches WHERE action_type = 'purchase'; + +-- Query: How many users added to cart but did NOT purchase? +SELECT theta_sketch_estimate( + theta_difference( + (SELECT user_sketch FROM action_sketches WHERE action_type = 'add_to_cart'), + (SELECT user_sketch FROM action_sketches WHERE action_type = 'purchase') + ) +) as cart_abandoners; + +-- Query: How many users both viewed pages AND purchased (intersection)? +SELECT theta_sketch_estimate( + theta_intersection( + (SELECT user_sketch FROM action_sketches WHERE action_type = 'page_view'), + (SELECT user_sketch FROM action_sketches WHERE action_type = 'purchase') + ) +) as engaged_purchasers; +``` + +### Example: Finding Trending Items with Top-K Sketches + +Track the most frequently occurring items across batches. + +```sql +-- Create a table to store hourly top-k sketches +CREATE TABLE hourly_search_sketches ( + hour_ts TIMESTAMP, + search_sketch STRUCT +) USING PARQUET; + +-- Process each hour's search queries +INSERT INTO hourly_search_sketches +SELECT + DATE_TRUNC('hour', search_time) as hour_ts, + approx_top_k_accumulate(search_term, 10000) as search_sketch +FROM search_logs +GROUP BY DATE_TRUNC('hour', search_time); + +-- Query: Get top 10 searches for a specific hour +SELECT approx_top_k_estimate(search_sketch, 10) as top_searches +FROM hourly_search_sketches +WHERE hour_ts = TIMESTAMP'2024-01-15 14:00:00'; + +-- Query: Get top 10 searches across the full day by combining sketches +SELECT approx_top_k_estimate( + approx_top_k_combine(search_sketch, 10000), + 10 +) as daily_top_searches +FROM hourly_search_sketches +WHERE DATE(hour_ts) = DATE'2024-01-15'; +``` diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index 5fed5c8d6719d..9fcdac38e7d5a 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -461,6 +461,9 @@ Aggregate Functions kll_sketch_agg_bigint kll_sketch_agg_double kll_sketch_agg_float + kll_merge_agg_bigint + kll_merge_agg_float + kll_merge_agg_double kurtosis last last_value diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 69706398253c0..a2db7e172b5da 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -4571,6 +4571,48 @@ def kll_sketch_agg_double( kll_sketch_agg_double.__doc__ = pysparkfuncs.kll_sketch_agg_double.__doc__ +def kll_merge_agg_bigint( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + fn = "kll_merge_agg_bigint" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +kll_merge_agg_bigint.__doc__ = pysparkfuncs.kll_merge_agg_bigint.__doc__ + + +def kll_merge_agg_float( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + fn = "kll_merge_agg_float" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +kll_merge_agg_float.__doc__ = pysparkfuncs.kll_merge_agg_float.__doc__ + + +def kll_merge_agg_double( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + fn = "kll_merge_agg_double" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +kll_merge_agg_double.__doc__ = pysparkfuncs.kll_merge_agg_double.__doc__ + + def kll_sketch_to_string_bigint(col: "ColumnOrName") -> Column: fn = "kll_sketch_to_string_bigint" return _invoke_function_over_columns(fn, col) diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index 6bbc69dc9bf65..64446a835d842 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -375,6 +375,9 @@ "kll_sketch_agg_bigint", "kll_sketch_agg_double", "kll_sketch_agg_float", + "kll_merge_agg_bigint", + "kll_merge_agg_double", + "kll_merge_agg_float", "kurtosis", "last", "last_value", diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5bb1b2d8b5efc..63b4ad64b5792 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -26694,6 +26694,138 @@ def kll_sketch_agg_double( return _invoke_function_over_columns(fn, col, lit(k)) +@_try_remote_functions +def kll_merge_agg_bigint( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + """ + Aggregate function: merges binary KllLongsSketch representations and returns the + merged sketch. The optional k parameter controls the size and accuracy of the merged + sketch (range 8-65535). If k is not specified, the merged sketch adopts the k value + from the first input sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The column containing binary KllLongsSketch representations + k : :class:`~pyspark.sql.Column` or int, optional + The k parameter that controls size and accuracy (range 8-65535) + + Returns + ------- + :class:`~pyspark.sql.Column` + The merged binary representation of the KllLongsSketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df1 = spark.createDataFrame([1,2,3], "INT") + >>> df2 = spark.createDataFrame([4,5,6], "INT") + >>> sketch1 = df1.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> sketch2 = df2.agg(sf.kll_sketch_agg_bigint("value").alias("sketch")) + >>> merged = sketch1.union(sketch2).agg(sf.kll_merge_agg_bigint("sketch").alias("merged")) + >>> n = merged.select(sf.kll_sketch_get_n_bigint("merged")).first()[0] + >>> n + 6 + """ + fn = "kll_merge_agg_bigint" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +@_try_remote_functions +def kll_merge_agg_float( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + """ + Aggregate function: merges binary KllFloatsSketch representations and returns the + merged sketch. The optional k parameter controls the size and accuracy of the merged + sketch (range 8-65535). If k is not specified, the merged sketch adopts the k value + from the first input sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The column containing binary KllFloatsSketch representations + k : :class:`~pyspark.sql.Column` or int, optional + The k parameter that controls size and accuracy (range 8-65535) + + Returns + ------- + :class:`~pyspark.sql.Column` + The merged binary representation of the KllFloatsSketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df1 = spark.createDataFrame([1.0,2.0,3.0], "FLOAT") + >>> df2 = spark.createDataFrame([4.0,5.0,6.0], "FLOAT") + >>> sketch1 = df1.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> sketch2 = df2.agg(sf.kll_sketch_agg_float("value").alias("sketch")) + >>> merged = sketch1.union(sketch2).agg(sf.kll_merge_agg_float("sketch").alias("merged")) + >>> n = merged.select(sf.kll_sketch_get_n_float("merged")).first()[0] + >>> n + 6 + """ + fn = "kll_merge_agg_float" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + +@_try_remote_functions +def kll_merge_agg_double( + col: "ColumnOrName", + k: Optional[Union[int, Column]] = None, +) -> Column: + """ + Aggregate function: merges binary KllDoublesSketch representations and returns the + merged sketch. The optional k parameter controls the size and accuracy of the merged + sketch (range 8-65535). If k is not specified, the merged sketch adopts the k value + from the first input sketch. + + .. versionadded:: 4.1.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + The column containing binary KllDoublesSketch representations + k : :class:`~pyspark.sql.Column` or int, optional + The k parameter that controls size and accuracy (range 8-65535) + + Returns + ------- + :class:`~pyspark.sql.Column` + The merged binary representation of the KllDoublesSketch. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df1 = spark.createDataFrame([1.0,2.0,3.0], "DOUBLE") + >>> df2 = spark.createDataFrame([4.0,5.0,6.0], "DOUBLE") + >>> sketch1 = df1.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> sketch2 = df2.agg(sf.kll_sketch_agg_double("value").alias("sketch")) + >>> merged = sketch1.union(sketch2).agg(sf.kll_merge_agg_double("sketch").alias("merged")) + >>> n = merged.select(sf.kll_sketch_get_n_double("merged")).first()[0] + >>> n + 6 + """ + fn = "kll_merge_agg_double" + if k is None: + return _invoke_function_over_columns(fn, col) + else: + return _invoke_function_over_columns(fn, col, lit(k)) + + @_try_remote_functions def kll_sketch_to_string_bigint(col: "ColumnOrName") -> Column: """ diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index b6d4e3a0547b3..a776fa6e80b74 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -2270,6 +2270,97 @@ def test_kll_sketch_with_nulls(self): # Should only count non-null values self.assertEqual(n, 3) + def test_kll_merge_agg_bigint(self): + """Test kll_merge_agg_bigint function""" + df1 = self.spark.createDataFrame([1, 2, 3], "INT") + df2 = self.spark.createDataFrame([4, 5, 6], "INT") + + sketch1 = df1.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + sketch2 = df2.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + # Union and merge sketches + merged = sketch1.union(sketch2).agg(F.kll_merge_agg_bigint("sketch").alias("merged")) + + # Verify the merged sketch contains all values + n = merged.select(F.kll_sketch_get_n_bigint("merged")).first()[0] + self.assertEqual(n, 6) + + # Test with explicit k parameter + merged_with_k = sketch1.union(sketch2).agg( + F.kll_merge_agg_bigint("sketch", 400).alias("merged") + ) + self.assertIsNotNone(merged_with_k.first()[0]) + + def test_kll_merge_agg_float(self): + """Test kll_merge_agg_float function""" + df1 = self.spark.createDataFrame([1.0, 2.0, 3.0], "FLOAT") + df2 = self.spark.createDataFrame([4.0, 5.0, 6.0], "FLOAT") + + sketch1 = df1.agg(F.kll_sketch_agg_float("value").alias("sketch")) + sketch2 = df2.agg(F.kll_sketch_agg_float("value").alias("sketch")) + + # Union and merge sketches + merged = sketch1.union(sketch2).agg(F.kll_merge_agg_float("sketch").alias("merged")) + + # Verify the merged sketch contains all values + n = merged.select(F.kll_sketch_get_n_float("merged")).first()[0] + self.assertEqual(n, 6) + + # Test with explicit k parameter + merged_with_k = sketch1.union(sketch2).agg( + F.kll_merge_agg_float("sketch", 300).alias("merged") + ) + self.assertIsNotNone(merged_with_k.first()[0]) + + def test_kll_merge_agg_double(self): + """Test kll_merge_agg_double function""" + df1 = self.spark.createDataFrame([1.0, 2.0, 3.0], "DOUBLE") + df2 = self.spark.createDataFrame([4.0, 5.0, 6.0], "DOUBLE") + + sketch1 = df1.agg(F.kll_sketch_agg_double("value").alias("sketch")) + sketch2 = df2.agg(F.kll_sketch_agg_double("value").alias("sketch")) + + # Union and merge sketches + merged = sketch1.union(sketch2).agg(F.kll_merge_agg_double("sketch").alias("merged")) + + # Verify the merged sketch contains all values + n = merged.select(F.kll_sketch_get_n_double("merged")).first()[0] + self.assertEqual(n, 6) + + # Test quantile on merged sketch + quantile = merged.select(F.kll_sketch_get_quantile_double("merged", F.lit(0.5))).first()[0] + self.assertIsNotNone(quantile) + + def test_kll_merge_agg_with_different_k(self): + """Test kll_merge_agg with different k values""" + df1 = self.spark.createDataFrame([1, 2, 3], "INT") + df2 = self.spark.createDataFrame([4, 5, 6], "INT") + + # Create sketches with different k values + sketch1 = df1.agg(F.kll_sketch_agg_bigint("value", 200).alias("sketch")) + sketch2 = df2.agg(F.kll_sketch_agg_bigint("value", 400).alias("sketch")) + + # Merge sketches with different k values (should adopt from first sketch) + merged = sketch1.union(sketch2).agg(F.kll_merge_agg_bigint("sketch").alias("merged")) + + n = merged.select(F.kll_sketch_get_n_bigint("merged")).first()[0] + self.assertEqual(n, 6) + + def test_kll_merge_agg_with_nulls(self): + """Test kll_merge_agg with null values""" + df1 = self.spark.createDataFrame([1, 2, 3], "INT") + df2 = self.spark.createDataFrame([4, None, 6], "INT") + + sketch1 = df1.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + sketch2 = df2.agg(F.kll_sketch_agg_bigint("value").alias("sketch")) + + # Merge sketches - null values should be ignored + merged = sketch1.union(sketch2).agg(F.kll_merge_agg_bigint("sketch").alias("merged")) + + n = merged.select(F.kll_sketch_get_n_bigint("merged")).first()[0] + # Should have 5 values (1,2,3,4,6 - null is ignored) + self.assertEqual(n, 5) + def test_datetime_functions(self): df = self.spark.range(1).selectExpr("'2017-01-22' as dateCol") parse_result = df.select(F.to_date(F.col("dateCol"))).first() diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index 0a14491de2238..d5d6481073908 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -1463,145 +1463,168 @@ object functions { kll_sketch_agg_double(Column(columnName)) /** - * Returns a string with human readable summary information about the KLL bigint sketch. + * Aggregate function: merges binary KllLongsSketch representations and returns the merged + * sketch. The optional k parameter controls the size and accuracy of the merged sketch (range + * 8-65535). If k is not specified, the merged sketch adopts the k value from the first input + * sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_to_string_bigint(e: Column): Column = - Column.fn("kll_sketch_to_string_bigint", e) + def kll_merge_agg_bigint(e: Column, k: Column): Column = + Column.fn("kll_merge_agg_bigint", e, k) /** - * Returns a string with human readable summary information about the KLL float sketch. + * Aggregate function: merges binary KllLongsSketch representations and returns the merged + * sketch. The optional k parameter controls the size and accuracy of the merged sketch (range + * 8-65535). If k is not specified, the merged sketch adopts the k value from the first input + * sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_to_string_float(e: Column): Column = - Column.fn("kll_sketch_to_string_float", e) + def kll_merge_agg_bigint(e: Column, k: Int): Column = + Column.fn("kll_merge_agg_bigint", e, lit(k)) /** - * Returns a string with human readable summary information about the KLL double sketch. + * Aggregate function: merges binary KllLongsSketch representations and returns the merged + * sketch. The optional k parameter controls the size and accuracy of the merged sketch (range + * 8-65535). If k is not specified, the merged sketch adopts the k value from the first input + * sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_to_string_double(e: Column): Column = - Column.fn("kll_sketch_to_string_double", e) + def kll_merge_agg_bigint(columnName: String, k: Int): Column = + kll_merge_agg_bigint(Column(columnName), k) /** - * Returns the number of items collected in the KLL bigint sketch. + * Aggregate function: merges binary KllLongsSketch representations and returns the merged + * sketch. If k is not specified, the merged sketch adopts the k value from the first input + * sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_n_bigint(e: Column): Column = - Column.fn("kll_sketch_get_n_bigint", e) + def kll_merge_agg_bigint(e: Column): Column = + Column.fn("kll_merge_agg_bigint", e) /** - * Returns the number of items collected in the KLL float sketch. + * Aggregate function: merges binary KllLongsSketch representations and returns the merged + * sketch. If k is not specified, the merged sketch adopts the k value from the first input + * sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_n_float(e: Column): Column = - Column.fn("kll_sketch_get_n_float", e) + def kll_merge_agg_bigint(columnName: String): Column = + kll_merge_agg_bigint(Column(columnName)) /** - * Returns the number of items collected in the KLL double sketch. + * Aggregate function: merges binary KllFloatsSketch representations and returns merged sketch. + * The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_n_double(e: Column): Column = - Column.fn("kll_sketch_get_n_double", e) + def kll_merge_agg_float(e: Column, k: Column): Column = + Column.fn("kll_merge_agg_float", e, k) /** - * Merges two KLL bigint sketch buffers together into one. + * Aggregate function: merges binary KllFloatsSketch representations and returns merged sketch. + * The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_merge_bigint(left: Column, right: Column): Column = - Column.fn("kll_sketch_merge_bigint", left, right) + def kll_merge_agg_float(e: Column, k: Int): Column = + Column.fn("kll_merge_agg_float", e, lit(k)) /** - * Merges two KLL float sketch buffers together into one. + * Aggregate function: merges binary KllFloatsSketch representations and returns merged sketch. + * The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_merge_float(left: Column, right: Column): Column = - Column.fn("kll_sketch_merge_float", left, right) + def kll_merge_agg_float(columnName: String, k: Int): Column = + kll_merge_agg_float(Column(columnName), k) /** - * Merges two KLL double sketch buffers together into one. + * Aggregate function: merges binary KllFloatsSketch representations and returns merged sketch. + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_merge_double(left: Column, right: Column): Column = - Column.fn("kll_sketch_merge_double", left, right) + def kll_merge_agg_float(e: Column): Column = + Column.fn("kll_merge_agg_float", e) /** - * Extracts a quantile value from a KLL bigint sketch given an input rank value. The rank can be - * a single value or an array. + * Aggregate function: merges binary KllFloatsSketch representations and returns merged sketch. + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_quantile_bigint(sketch: Column, rank: Column): Column = - Column.fn("kll_sketch_get_quantile_bigint", sketch, rank) + def kll_merge_agg_float(columnName: String): Column = + kll_merge_agg_float(Column(columnName)) /** - * Extracts a quantile value from a KLL float sketch given an input rank value. The rank can be - * a single value or an array. + * Aggregate function: merges binary KllDoublesSketch representations and returns merged sketch. + * The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_quantile_float(sketch: Column, rank: Column): Column = - Column.fn("kll_sketch_get_quantile_float", sketch, rank) + def kll_merge_agg_double(e: Column, k: Column): Column = + Column.fn("kll_merge_agg_double", e, k) /** - * Extracts a quantile value from a KLL double sketch given an input rank value. The rank can be - * a single value or an array. + * Aggregate function: merges binary KllDoublesSketch representations and returns merged sketch. + * The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_quantile_double(sketch: Column, rank: Column): Column = - Column.fn("kll_sketch_get_quantile_double", sketch, rank) + def kll_merge_agg_double(e: Column, k: Int): Column = + Column.fn("kll_merge_agg_double", e, lit(k)) /** - * Extracts a rank value from a KLL bigint sketch given an input quantile value. The quantile - * can be a single value or an array. + * Aggregate function: merges binary KllDoublesSketch representations and returns merged sketch. + * The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_rank_bigint(sketch: Column, quantile: Column): Column = - Column.fn("kll_sketch_get_rank_bigint", sketch, quantile) + def kll_merge_agg_double(columnName: String, k: Int): Column = + kll_merge_agg_double(Column(columnName), k) /** - * Extracts a rank value from a KLL float sketch given an input quantile value. The quantile can - * be a single value or an array. + * Aggregate function: merges binary KllDoublesSketch representations and returns merged sketch. + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_rank_float(sketch: Column, quantile: Column): Column = - Column.fn("kll_sketch_get_rank_float", sketch, quantile) + def kll_merge_agg_double(e: Column): Column = + Column.fn("kll_merge_agg_double", e) /** - * Extracts a rank value from a KLL double sketch given an input quantile value. The quantile - * can be a single value or an array. + * Aggregate function: merges binary KllDoublesSketch representations and returns merged sketch. + * If k is not specified, the merged sketch adopts the k value from the first input sketch. * - * @group misc_funcs + * @group agg_funcs * @since 4.1.0 */ - def kll_sketch_get_rank_double(sketch: Column, quantile: Column): Column = - Column.fn("kll_sketch_get_rank_double", sketch, quantile) + def kll_merge_agg_double(columnName: String): Column = + kll_merge_agg_double(Column(columnName)) /** * Aggregate function: returns the concatenation of non-null input values. @@ -4109,6 +4132,147 @@ object functions { def theta_union(c1: Column, c2: Column, lgNomEntries: Column): Column = Column.fn("theta_union", c1, c2, lgNomEntries) + /** + * Returns a string with human readable summary information about the KLL bigint sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_to_string_bigint(e: Column): Column = + Column.fn("kll_sketch_to_string_bigint", e) + + /** + * Returns a string with human readable summary information about the KLL float sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_to_string_float(e: Column): Column = + Column.fn("kll_sketch_to_string_float", e) + + /** + * Returns a string with human readable summary information about the KLL double sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_to_string_double(e: Column): Column = + Column.fn("kll_sketch_to_string_double", e) + + /** + * Returns the number of items collected in the KLL bigint sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_n_bigint(e: Column): Column = + Column.fn("kll_sketch_get_n_bigint", e) + + /** + * Returns the number of items collected in the KLL float sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_n_float(e: Column): Column = + Column.fn("kll_sketch_get_n_float", e) + + /** + * Returns the number of items collected in the KLL double sketch. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_n_double(e: Column): Column = + Column.fn("kll_sketch_get_n_double", e) + + /** + * Merges two KLL bigint sketch buffers together into one. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_merge_bigint(left: Column, right: Column): Column = + Column.fn("kll_sketch_merge_bigint", left, right) + + /** + * Merges two KLL float sketch buffers together into one. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_merge_float(left: Column, right: Column): Column = + Column.fn("kll_sketch_merge_float", left, right) + + /** + * Merges two KLL double sketch buffers together into one. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_merge_double(left: Column, right: Column): Column = + Column.fn("kll_sketch_merge_double", left, right) + + /** + * Extracts a quantile value from a KLL bigint sketch given an input rank value. The rank can be + * a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_quantile_bigint(sketch: Column, rank: Column): Column = + Column.fn("kll_sketch_get_quantile_bigint", sketch, rank) + + /** + * Extracts a quantile value from a KLL float sketch given an input rank value. The rank can be + * a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_quantile_float(sketch: Column, rank: Column): Column = + Column.fn("kll_sketch_get_quantile_float", sketch, rank) + + /** + * Extracts a quantile value from a KLL double sketch given an input rank value. The rank can be + * a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_quantile_double(sketch: Column, rank: Column): Column = + Column.fn("kll_sketch_get_quantile_double", sketch, rank) + + /** + * Extracts a rank value from a KLL bigint sketch given an input quantile value. The quantile + * can be a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_rank_bigint(sketch: Column, quantile: Column): Column = + Column.fn("kll_sketch_get_rank_bigint", sketch, quantile) + + /** + * Extracts a rank value from a KLL float sketch given an input quantile value. The quantile can + * be a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_rank_float(sketch: Column, quantile: Column): Column = + Column.fn("kll_sketch_get_rank_float", sketch, quantile) + + /** + * Extracts a rank value from a KLL double sketch given an input quantile value. The quantile + * can be a single value or an array. + * + * @group misc_funcs + * @since 4.1.0 + */ + def kll_sketch_get_rank_double(sketch: Column, quantile: Column): Column = + Column.fn("kll_sketch_get_rank_double", sketch, quantile) + /** * Returns the user name of current execution context. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 36e40306be7d6..e1ca5ad918476 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -539,21 +539,9 @@ object FunctionRegistry { expression[KllSketchAggBigint]("kll_sketch_agg_bigint"), expression[KllSketchAggFloat]("kll_sketch_agg_float"), expression[KllSketchAggDouble]("kll_sketch_agg_double"), - expression[KllSketchToStringBigint]("kll_sketch_to_string_bigint"), - expression[KllSketchToStringFloat]("kll_sketch_to_string_float"), - expression[KllSketchToStringDouble]("kll_sketch_to_string_double"), - expression[KllSketchGetNBigint]("kll_sketch_get_n_bigint"), - expression[KllSketchGetNFloat]("kll_sketch_get_n_float"), - expression[KllSketchGetNDouble]("kll_sketch_get_n_double"), - expression[KllSketchMergeBigint]("kll_sketch_merge_bigint"), - expression[KllSketchMergeFloat]("kll_sketch_merge_float"), - expression[KllSketchMergeDouble]("kll_sketch_merge_double"), - expression[KllSketchGetQuantileBigint]("kll_sketch_get_quantile_bigint"), - expression[KllSketchGetQuantileFloat]("kll_sketch_get_quantile_float"), - expression[KllSketchGetQuantileDouble]("kll_sketch_get_quantile_double"), - expression[KllSketchGetRankBigint]("kll_sketch_get_rank_bigint"), - expression[KllSketchGetRankFloat]("kll_sketch_get_rank_float"), - expression[KllSketchGetRankDouble]("kll_sketch_get_rank_double"), + expression[KllMergeAggBigint]("kll_merge_agg_bigint"), + expression[KllMergeAggFloat]("kll_merge_agg_float"), + expression[KllMergeAggDouble]("kll_merge_agg_double"), // string functions expression[Ascii]("ascii"), @@ -819,6 +807,21 @@ object FunctionRegistry { expression[ThetaDifference]("theta_difference"), expression[ThetaIntersection]("theta_intersection"), expression[ApproxTopKEstimate]("approx_top_k_estimate"), + expression[KllSketchToStringBigint]("kll_sketch_to_string_bigint"), + expression[KllSketchToStringFloat]("kll_sketch_to_string_float"), + expression[KllSketchToStringDouble]("kll_sketch_to_string_double"), + expression[KllSketchGetNBigint]("kll_sketch_get_n_bigint"), + expression[KllSketchGetNFloat]("kll_sketch_get_n_float"), + expression[KllSketchGetNDouble]("kll_sketch_get_n_double"), + expression[KllSketchMergeBigint]("kll_sketch_merge_bigint"), + expression[KllSketchMergeFloat]("kll_sketch_merge_float"), + expression[KllSketchMergeDouble]("kll_sketch_merge_double"), + expression[KllSketchGetQuantileBigint]("kll_sketch_get_quantile_bigint"), + expression[KllSketchGetQuantileFloat]("kll_sketch_get_quantile_float"), + expression[KllSketchGetQuantileDouble]("kll_sketch_get_quantile_double"), + expression[KllSketchGetRankBigint]("kll_sketch_get_rank_bigint"), + expression[KllSketchGetRankFloat]("kll_sketch_get_rank_float"), + expression[KllSketchGetRankDouble]("kll_sketch_get_rank_double"), // grouping sets expression[Grouping]("grouping"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala index e74b22219cf61..6e3ea19425d9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/kllAggregates.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.datasketches.kll.{KllDoublesSketch, KllFloatsSketch, KllLongsSketch} +import org.apache.datasketches.kll.{KllDoublesSketch, KllFloatsSketch, KllLongsSketch, KllSketch} import org.apache.datasketches.memory.Memory import org.apache.spark.SparkUnsupportedOperationException @@ -108,9 +108,10 @@ case class KllSketchAggBigint( override def checkInputDataTypes(): TypeCheckResult = { val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { - return defaultCheck + defaultCheck + } else { + checkKInputDataTypes() } - checkKInputDataTypes() } override def createAggregationBuffer(): KllLongsSketch = @@ -123,26 +124,25 @@ case class KllSketchAggBigint( * Note, null values are ignored. */ override def update(sketch: KllLongsSketch, input: InternalRow): KllLongsSketch = { - // Return early for null values. val v = child.eval(input) if (v == null) { - return sketch - } - // Handle the different data types for sketch updates. - child.dataType match { - case ByteType => - sketch.update(v.asInstanceOf[Byte].toLong) - case IntegerType => - sketch.update(v.asInstanceOf[Int].toLong) - case LongType => - sketch.update(v.asInstanceOf[Long]) - case ShortType => - sketch.update(v.asInstanceOf[Short].toLong) - case _ => - throw unexpectedInputDataTypeError(child) + sketch + } else { + // Handle the different data types for sketch updates. + child.dataType match { + case ByteType => + sketch.update(v.asInstanceOf[Byte].toLong) + case IntegerType => + sketch.update(v.asInstanceOf[Int].toLong) + case LongType => + sketch.update(v.asInstanceOf[Long]) + case ShortType => + sketch.update(v.asInstanceOf[Short].toLong) + case _ => + throw unexpectedInputDataTypeError(child) + } + sketch } - - sketch } /** Merges an input sketch into the current aggregation buffer. */ @@ -250,9 +250,10 @@ case class KllSketchAggFloat( override def checkInputDataTypes(): TypeCheckResult = { val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { - return defaultCheck + defaultCheck + } else { + checkKInputDataTypes() } - checkKInputDataTypes() } override def createAggregationBuffer(): KllFloatsSketch = @@ -265,20 +266,19 @@ case class KllSketchAggFloat( * Note, Null values are ignored. */ override def update(sketch: KllFloatsSketch, input: InternalRow): KllFloatsSketch = { - // Return early for null values. val v = child.eval(input) if (v == null) { - return sketch - } - // Handle the different data types for sketch updates. - child.dataType match { - case FloatType => - sketch.update(v.asInstanceOf[Float]) - case _ => - throw unexpectedInputDataTypeError(child) + sketch + } else { + // Handle the different data types for sketch updates. + child.dataType match { + case FloatType => + sketch.update(v.asInstanceOf[Float]) + case _ => + throw unexpectedInputDataTypeError(child) + } + sketch } - - sketch } /** Merges an input sketch into the current aggregation buffer. */ @@ -386,9 +386,10 @@ case class KllSketchAggDouble( override def checkInputDataTypes(): TypeCheckResult = { val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { - return defaultCheck + defaultCheck + } else { + checkKInputDataTypes() } - checkKInputDataTypes() } override def createAggregationBuffer(): KllDoublesSketch = @@ -401,22 +402,21 @@ case class KllSketchAggDouble( * Note, Null values are ignored. */ override def update(sketch: KllDoublesSketch, input: InternalRow): KllDoublesSketch = { - // Return early for null values. val v = child.eval(input) if (v == null) { - return sketch - } - // Handle the different data types for sketch updates. - child.dataType match { - case DoubleType => - sketch.update(v.asInstanceOf[Double]) - case FloatType => - sketch.update(v.asInstanceOf[Float].toDouble) - case _ => - throw unexpectedInputDataTypeError(child) + sketch + } else { + // Handle the different data types for sketch updates. + child.dataType match { + case DoubleType => + sketch.update(v.asInstanceOf[Double]) + case FloatType => + sketch.update(v.asInstanceOf[Float].toDouble) + case _ => + throw unexpectedInputDataTypeError(child) + } + sketch } - - sketch } /** Merges an input sketch into the current aggregation buffer. */ @@ -449,6 +449,350 @@ case class KllSketchAggDouble( } } +/** + * The KllMergeAggBigint function merges multiple Apache DataSketches KllLongsSketch instances + * that have been serialized to binary format. This is useful for combining sketches created + * in separate aggregations (e.g., from different partitions or time windows). + * It outputs the merged binary representation of the KllLongsSketch. + * + * See [[https://datasketches.apache.org/docs/KLL/KLLSketch.html]] for more information. + * + * @param child + * child expression containing binary KllLongsSketch representations to merge + * @param kExpr + * optional expression for the k parameter from the Apache DataSketches library that controls + * the size and accuracy of the sketch. Must be a constant integer between 8 and 65535. + * If not specified, the merged sketch adopts the k value from the first input sketch. + * If specified, the value is used to initialize the aggregation buffer. The merge operation + * can handle input sketches with different k values. Larger k values provide more accurate + * estimates but result in larger, slower sketches. + * @param mutableAggBufferOffset + * offset for mutable aggregation buffer + * @param inputAggBufferOffset + * offset for input aggregation buffer + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, k]) - Merges binary KllLongsSketch representations and returns the merged sketch. + The input expression should contain binary sketch representations (e.g., from kll_sketch_agg_bigint). + The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + If k is not specified, the merged sketch adopts the k value from the first input sketch. + """, + examples = """ + Examples: + > SELECT kll_sketch_get_n_bigint(_FUNC_(sketch)) FROM (SELECT kll_sketch_agg_bigint(col) as sketch FROM VALUES (1), (2), (3) tab(col) UNION ALL SELECT kll_sketch_agg_bigint(col) as sketch FROM VALUES (4), (5), (6) tab(col)) t; + 6 + """, + group = "agg_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class KllMergeAggBigint( + child: Expression, + kExpr: Option[Expression] = None, + override val mutableAggBufferOffset: Int = 0, + override val inputAggBufferOffset: Int = 0) + extends KllMergeAggBase[KllLongsSketch] { + def this(child: Expression) = this(child, None, 0, 0) + def this(child: Expression, kExpr: Expression) = this(child, Some(kExpr), 0, 0) + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): KllMergeAggBigint = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + override def withNewInputAggBufferOffset( + newInputAggBufferOffset: Int): KllMergeAggBigint = + copy(inputAggBufferOffset = newInputAggBufferOffset) + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): KllMergeAggBigint = { + if (newChildren.length == 1) { + copy(child = newChildren(0), kExpr = None) + } else { + copy(child = newChildren(0), kExpr = Some(newChildren(1))) + } + } + + override def prettyName: String = "kll_merge_agg_bigint" + + // Factory method implementations + protected def newHeapInstance(k: Int): KllLongsSketch = KllLongsSketch.newHeapInstance(k) + protected def wrapSketch(bytes: Array[Byte]): KllLongsSketch = + KllLongsSketch.wrap(Memory.wrap(bytes)) + protected def heapifySketch(bytes: Array[Byte]): KllLongsSketch = + KllLongsSketch.heapify(Memory.wrap(bytes)) + protected def toByteArray(sketch: KllLongsSketch): Array[Byte] = sketch.toByteArray +} + +/** + * The KllMergeAggFloat function merges multiple Apache DataSketches KllFloatsSketch instances + * that have been serialized to binary format. This is useful for combining sketches created + * in separate aggregations (e.g., from different partitions or time windows). + * It outputs the merged binary representation of the KllFloatsSketch. + * + * See [[https://datasketches.apache.org/docs/KLL/KLLSketch.html]] for more information. + * + * @param child + * child expression containing binary KllFloatsSketch representations to merge + * @param kExpr + * optional expression for the k parameter from the Apache DataSketches library that controls + * the size and accuracy of the sketch. Must be a constant integer between 8 and 65535. + * If not specified, the merged sketch adopts the k value from the first input sketch. + * If specified, the value is used to initialize the aggregation buffer. The merge operation + * can handle input sketches with different k values. Larger k values provide more accurate + * estimates but result in larger, slower sketches. + * @param mutableAggBufferOffset + * offset for mutable aggregation buffer + * @param inputAggBufferOffset + * offset for input aggregation buffer + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, k]) - Merges binary KllFloatsSketch representations and returns the merged sketch. + The input expression should contain binary sketch representations (e.g., from kll_sketch_agg_float). + The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + If k is not specified, the merged sketch adopts the k value from the first input sketch. + """, + examples = """ + Examples: + > SELECT kll_sketch_get_n_float(_FUNC_(sketch)) FROM (SELECT kll_sketch_agg_float(col) as sketch FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)) tab(col) UNION ALL SELECT kll_sketch_agg_float(col) as sketch FROM VALUES (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)), (CAST(6.0 AS FLOAT)) tab(col)) t; + 6 + """, + group = "agg_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class KllMergeAggFloat( + child: Expression, + kExpr: Option[Expression] = None, + override val mutableAggBufferOffset: Int = 0, + override val inputAggBufferOffset: Int = 0) + extends KllMergeAggBase[KllFloatsSketch] { + def this(child: Expression) = this(child, None, 0, 0) + def this(child: Expression, kExpr: Expression) = this(child, Some(kExpr), 0, 0) + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): KllMergeAggFloat = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + override def withNewInputAggBufferOffset( + newInputAggBufferOffset: Int): KllMergeAggFloat = + copy(inputAggBufferOffset = newInputAggBufferOffset) + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): KllMergeAggFloat = { + if (newChildren.length == 1) { + copy(child = newChildren(0), kExpr = None) + } else { + copy(child = newChildren(0), kExpr = Some(newChildren(1))) + } + } + + override def prettyName: String = "kll_merge_agg_float" + + // Factory method implementations + protected def newHeapInstance(k: Int): KllFloatsSketch = KllFloatsSketch.newHeapInstance(k) + protected def wrapSketch(bytes: Array[Byte]): KllFloatsSketch = + KllFloatsSketch.wrap(Memory.wrap(bytes)) + protected def heapifySketch(bytes: Array[Byte]): KllFloatsSketch = + KllFloatsSketch.heapify(Memory.wrap(bytes)) + protected def toByteArray(sketch: KllFloatsSketch): Array[Byte] = sketch.toByteArray +} + +/** + * The KllMergeAggDouble function merges multiple Apache DataSketches KllDoublesSketch instances + * that have been serialized to binary format. This is useful for combining sketches created + * in separate aggregations (e.g., from different partitions or time windows). + * It outputs the merged binary representation of the KllDoublesSketch. + * + * See [[https://datasketches.apache.org/docs/KLL/KLLSketch.html]] for more information. + * + * @param child + * child expression containing binary KllDoublesSketch representations to merge + * @param kExpr + * optional expression for the k parameter from the Apache DataSketches library that controls + * the size and accuracy of the sketch. Must be a constant integer between 8 and 65535. + * If not specified, the merged sketch adopts the k value from the first input sketch. + * If specified, the value is used to initialize the aggregation buffer. The merge operation + * can handle input sketches with different k values. Larger k values provide more accurate + * estimates but result in larger, slower sketches. + * @param mutableAggBufferOffset + * offset for mutable aggregation buffer + * @param inputAggBufferOffset + * offset for input aggregation buffer + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, k]) - Merges binary KllDoublesSketch representations and returns the merged sketch. + The input expression should contain binary sketch representations (e.g., from kll_sketch_agg_double). + The optional k parameter controls the size and accuracy of the merged sketch (range 8-65535). + If k is not specified, the merged sketch adopts the k value from the first input sketch. + """, + examples = """ + Examples: + > SELECT kll_sketch_get_n_double(_FUNC_(sketch)) FROM (SELECT kll_sketch_agg_double(col) as sketch FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)) tab(col) UNION ALL SELECT kll_sketch_agg_double(col) as sketch FROM VALUES (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)), (CAST(6.0 AS DOUBLE)) tab(col)) t; + 6 + """, + group = "agg_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class KllMergeAggDouble( + child: Expression, + kExpr: Option[Expression] = None, + override val mutableAggBufferOffset: Int = 0, + override val inputAggBufferOffset: Int = 0) + extends KllMergeAggBase[KllDoublesSketch] { + def this(child: Expression) = this(child, None, 0, 0) + def this(child: Expression, kExpr: Expression) = this(child, Some(kExpr), 0, 0) + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): KllMergeAggDouble = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + override def withNewInputAggBufferOffset( + newInputAggBufferOffset: Int): KllMergeAggDouble = + copy(inputAggBufferOffset = newInputAggBufferOffset) + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): KllMergeAggDouble = { + if (newChildren.length == 1) { + copy(child = newChildren(0), kExpr = None) + } else { + copy(child = newChildren(0), kExpr = Some(newChildren(1))) + } + } + + override def prettyName: String = "kll_merge_agg_double" + + // Factory method implementations + protected def newHeapInstance(k: Int): KllDoublesSketch = KllDoublesSketch.newHeapInstance(k) + protected def wrapSketch(bytes: Array[Byte]): KllDoublesSketch = + KllDoublesSketch.wrap(Memory.wrap(bytes)) + protected def heapifySketch(bytes: Array[Byte]): KllDoublesSketch = + KllDoublesSketch.heapify(Memory.wrap(bytes)) + protected def toByteArray(sketch: KllDoublesSketch): Array[Byte] = sketch.toByteArray +} + +/** + * Base abstract class for KLL merge aggregate functions that provides common implementation + * for merging serialized KLL sketches with optional k parameter. + * + * @tparam T The KLL sketch type (KllLongsSketch, KllFloatsSketch, or KllDoublesSketch) + */ +abstract class KllMergeAggBase[T <: KllSketch] + extends TypedImperativeAggregate[Option[T]] + with KllSketchAggBase + with ExpectsInputTypes { + + def child: Expression + + // Abstract factory methods for sketch-specific instantiation + protected def newHeapInstance(k: Int): T + protected def wrapSketch(bytes: Array[Byte]): T + protected def heapifySketch(bytes: Array[Byte]): T + protected def toByteArray(sketch: T): Array[Byte] + + // Common implementations for all merge aggregates + override def children: Seq[Expression] = child +: kExpr.toSeq + + override def dataType: DataType = BinaryType + + override def inputTypes: Seq[AbstractDataType] = { + val baseTypes = Seq(BinaryType) + if (kExpr.isDefined) baseTypes :+ IntegerType else baseTypes + } + + override def nullable: Boolean = false + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + defaultCheck + } else { + checkKInputDataTypes() + } + } + + /** + * Defer instantiation of the sketch instance until we've deserialized + * our first sketch (if kExpr was not provided), and use that sketch's k value. + * + * @return None if kExpr was not provided, otherwise Some(sketch with specified k) + */ + override def createAggregationBuffer(): Option[T] = { + if (kExpr.isDefined) { + Some(newHeapInstance(kValue)) + } else { + None + } + } + + /** + * Evaluate the input row and wrap the binary sketch, then merge it into + * the current aggregation buffer. + * Note, null values are ignored. + */ + override def update(sketchOption: Option[T], input: InternalRow): Option[T] = { + val v = child.eval(input) + if (v == null) { + sketchOption + } else { + try { + val sketchBytes = v.asInstanceOf[Array[Byte]] + val inputSketch = wrapSketch(sketchBytes) + val sketch = sketchOption.getOrElse(newHeapInstance(inputSketch.getK())) + sketch.merge(inputSketch) + Some(sketch) + } catch { + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) + } + } + } + + /** Merges an input sketch into the current aggregation buffer. */ + override def merge(updateBufferOption: Option[T], inputOption: Option[T]): Option[T] = { + (updateBufferOption, inputOption) match { + case (Some(updateBuffer), Some(input)) => + try { + updateBuffer.merge(input) + Some(updateBuffer) + } catch { + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) + } + case (Some(_), None) => updateBufferOption + case (None, Some(_)) => inputOption + case (None, None) => None + } + } + + /** Returns a sketch derived from the input column or expression. */ + override def eval(sketchOption: Option[T]): Any = { + sketchOption match { + case Some(sketch) => toByteArray(sketch) + case None => toByteArray(newHeapInstance(kValue)) + } + } + + /** Converts the underlying sketch state into a byte array. */ + override def serialize(sketchOption: Option[T]): Array[Byte] = { + sketchOption match { + case Some(sketch) => toByteArray(sketch) + case None => toByteArray(newHeapInstance(kValue)) + } + } + + /** Wraps the byte array into a sketch instance. */ + override def deserialize(buffer: Array[Byte]): Option[T] = { + if (buffer.nonEmpty) { + try { + Some(heapifySketch(buffer)) + } catch { + case _: Exception => + throw QueryExecutionErrors.kllInvalidInputSketchBuffer(prettyName) + } + } else { + createAggregationBuffer() + } + } +} + /** * Common trait for KLL sketch aggregate functions that support an optional k parameter. */ diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 7b6cbabeb1296..a5c98675c9773 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -455,6 +455,9 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg | hll_sketch_agg | SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HllUnionAgg | hll_union_agg | SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) FROM (SELECT hll_sketch_agg(col) as sketch FROM VALUES (1) tab(col) UNION ALL SELECT hll_sketch_agg(col, 20) as sketch FROM VALUES (1) tab(col)) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.KllMergeAggBigint | kll_merge_agg_bigint | SELECT kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch)) FROM (SELECT kll_sketch_agg_bigint(col) as sketch FROM VALUES (1), (2), (3) tab(col) UNION ALL SELECT kll_sketch_agg_bigint(col) as sketch FROM VALUES (4), (5), (6) tab(col)) t | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.KllMergeAggDouble | kll_merge_agg_double | SELECT kll_sketch_get_n_double(kll_merge_agg_double(sketch)) FROM (SELECT kll_sketch_agg_double(col) as sketch FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)) tab(col) UNION ALL SELECT kll_sketch_agg_double(col) as sketch FROM VALUES (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)), (CAST(6.0 AS DOUBLE)) tab(col)) t | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.KllMergeAggFloat | kll_merge_agg_float | SELECT kll_sketch_get_n_float(kll_merge_agg_float(sketch)) FROM (SELECT kll_sketch_agg_float(col) as sketch FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)) tab(col) UNION ALL SELECT kll_sketch_agg_float(col) as sketch FROM VALUES (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)), (CAST(6.0 AS FLOAT)) tab(col)) t | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.KllSketchAggBigint | kll_sketch_agg_bigint | SELECT LENGTH(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col))) > 0 FROM VALUES (1), (2), (3), (4), (5) tab(col) | struct<(length(kll_sketch_to_string_bigint(kll_sketch_agg_bigint(col))) > 0):boolean> | | org.apache.spark.sql.catalyst.expressions.aggregate.KllSketchAggDouble | kll_sketch_agg_double | SELECT LENGTH(kll_sketch_to_string_double(kll_sketch_agg_double(col))) > 0 FROM VALUES (CAST(1.0 AS DOUBLE)), (CAST(2.0 AS DOUBLE)), (CAST(3.0 AS DOUBLE)), (CAST(4.0 AS DOUBLE)), (CAST(5.0 AS DOUBLE)) tab(col) | struct<(length(kll_sketch_to_string_double(kll_sketch_agg_double(col))) > 0):boolean> | | org.apache.spark.sql.catalyst.expressions.aggregate.KllSketchAggFloat | kll_sketch_agg_float | SELECT LENGTH(kll_sketch_to_string_float(kll_sketch_agg_float(col))) > 0 FROM VALUES (CAST(1.0 AS FLOAT)), (CAST(2.0 AS FLOAT)), (CAST(3.0 AS FLOAT)), (CAST(4.0 AS FLOAT)), (CAST(5.0 AS FLOAT)) tab(col) | struct<(length(kll_sketch_to_string_float(kll_sketch_agg_float(col))) > 0):boolean> | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out index 3eea568420c04..8a2b50131627a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out @@ -326,6 +326,262 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT + parity, + kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) AS total_count +FROM ( + SELECT + col1 % 2 AS parity, + kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + GROUP BY col1 % 2 +) grouped_sketches +GROUP BY parity +HAVING kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) > 3 +-- !query analysis +Filter (total_count#xL > cast(3 as bigint)) ++- Aggregate [parity#x], [parity#x, kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col#x, None, 0, 0)) AS total_count#xL] + +- SubqueryAlias grouped_sketches + +- Aggregate [(col1#x % 2)], [(col1#x % 2) AS parity#x, kll_sketch_agg_bigint(col1#x, None, 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_int_1_5_through_7_11 + +- Relation spark_catalog.default.t_int_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + WHERE col1 > 1000 +) empty_sketches +-- !query analysis +Aggregate [kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col#x, None, 0, 0)) AS empty_merge_n#xL] ++- SubqueryAlias empty_sketches + +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS sketch_col#x] + +- Filter (col1#x > 1000) + +- SubqueryAlias spark_catalog.default.t_int_1_5_through_7_11 + +- Relation spark_catalog.default.t_int_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_float(kll_merge_agg_float(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 + WHERE col1 > 1000.0 +) empty_sketches +-- !query analysis +Aggregate [kll_sketch_get_n_float(kll_merge_agg_float(sketch_col#x, None, 0, 0)) AS empty_merge_n#xL] ++- SubqueryAlias empty_sketches + +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS sketch_col#x] + +- Filter (cast(col1#x as double) > cast(1000.0 as double)) + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_sketch_get_n_double(kll_merge_agg_double(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 + WHERE col1 > 1000.0 +) empty_sketches +-- !query analysis +Aggregate [kll_sketch_get_n_double(kll_merge_agg_double(sketch_col#x, None, 0, 0)) AS empty_merge_n#xL] ++- SubqueryAlias empty_sketches + +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS sketch_col#x] + +- Filter (col1#x > cast(1000.0 as double)) + +- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 + +- Relation spark_catalog.default.t_double_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_merge_agg_bigint(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_short_1_5_through_7_11 + ) sketches +) +-- !query analysis +Project [lower(kll_sketch_to_string_bigint(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_merge_agg_bigint(sketch_col#x, None, 0, 0) AS agg#x] + +- SubqueryAlias sketches + +- Union false, false + :- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS sketch_col#x] + : +- SubqueryAlias spark_catalog.default.t_int_1_5_through_7_11 + : +- Relation spark_catalog.default.t_int_1_5_through_7_11[col1#x,col2#x] parquet + +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_short_1_5_through_7_11 + +- Relation spark_catalog.default.t_short_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 5.5) < 1.0 AS median_close_to_5_5, + abs(kll_sketch_get_rank_float(agg, 5.0) - 0.35) < 0.15 AS rank5_close_to_0_35 +FROM ( + SELECT kll_merge_agg_float(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_float(col2) AS sketch_col + FROM t_float_1_5_through_7_11 + ) sketches +) +-- !query analysis +Project [lower(kll_sketch_to_string_float(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((cast(kll_sketch_get_quantile_float(agg#x, cast(0.5 as double)) as double) - cast(5.5 as double))) < cast(1.0 as double)) AS median_close_to_5_5#x, (abs((kll_sketch_get_rank_float(agg#x, cast(5.0 as float)) - cast(0.35 as double))) < cast(0.15 as double)) AS rank5_close_to_0_35#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_merge_agg_float(sketch_col#x, None, 0, 0) AS agg#x] + +- SubqueryAlias sketches + +- Union false, false + :- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS sketch_col#x] + : +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + : +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + +- Aggregate [kll_sketch_agg_float(col2#x, None, 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 6.0) < 1.0 AS median_close_to_6, + abs(kll_sketch_get_rank_double(agg, 5.0) - 0.35) < 0.15 AS rank5_close_to_0_35 +FROM ( + SELECT kll_merge_agg_double(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_double(col2) AS sketch_col + FROM t_float_1_5_through_7_11 + ) sketches +) +-- !query analysis +Project [lower(kll_sketch_to_string_double(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(6.0 as double))) < cast(1.0 as double)) AS median_close_to_6#x, (abs((kll_sketch_get_rank_double(agg#x, cast(5.0 as double)) - cast(0.35 as double))) < cast(0.15 as double)) AS rank5_close_to_0_35#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [kll_merge_agg_double(sketch_col#x, None, 0, 0) AS agg#x] + +- SubqueryAlias sketches + +- Union false, false + :- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS sketch_col#x] + : +- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 + : +- Relation spark_catalog.default.t_double_1_5_through_7_11[col1#x,col2#x] parquet + +- Aggregate [kll_sketch_agg_double(col2#x, None, 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_merge_agg_bigint(sketch_col, 400))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_bigint(col1, 400) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col2, 400) AS sketch_col + FROM t_byte_1_5_through_7_11 +) sketches +-- !query analysis +Aggregate [(length(kll_sketch_to_string_bigint(kll_merge_agg_bigint(sketch_col#x, Some(400), 0, 0))) > 0) AS merged_with_k#x] ++- SubqueryAlias sketches + +- Union false, false + :- Aggregate [kll_sketch_agg_bigint(col1#xL, Some(400), 0, 0) AS sketch_col#x] + : +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + : +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + +- Aggregate [kll_sketch_agg_bigint(col2#x, Some(400), 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 + +- Relation spark_catalog.default.t_byte_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_float(kll_merge_agg_float(sketch_col, 300))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_float(col1, 300) AS sketch_col + FROM t_float_1_5_through_7_11 +) sketches +-- !query analysis +Aggregate [(length(kll_sketch_to_string_float(kll_merge_agg_float(sketch_col#x, Some(300), 0, 0))) > 0) AS merged_with_k#x] ++- SubqueryAlias sketches + +- Aggregate [kll_sketch_agg_float(col1#x, Some(300), 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT LENGTH(kll_sketch_to_string_double(kll_merge_agg_double(sketch_col, 500))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_double(col1, 500) AS sketch_col + FROM t_double_1_5_through_7_11 +) sketches +-- !query analysis +Aggregate [(length(kll_sketch_to_string_double(kll_merge_agg_double(sketch_col#x, Some(500), 0, 0))) > 0) AS merged_with_k#x] ++- SubqueryAlias sketches + +- Aggregate [kll_sketch_agg_double(col1#x, Some(500), 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 + +- Relation spark_catalog.default.t_double_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match +FROM ( + SELECT kll_merge_agg_bigint(sketch_col) AS agg_with_nulls + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT CAST(NULL AS BINARY) AS sketch_col + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_byte_1_5_through_7_11 + ) sketches_with_nulls +) WITH_NULLS, +( + SELECT kll_merge_agg_bigint(sketch_col) AS agg_without_nulls + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_byte_1_5_through_7_11 + ) sketches_without_nulls +) WITHOUT_NULLS +-- !query analysis +Project [(abs((kll_sketch_get_quantile_bigint(agg_with_nulls#x, cast(0.5 as double)) - kll_sketch_get_quantile_bigint(agg_without_nulls#x, cast(0.5 as double)))) < cast(1 as bigint)) AS medians_match#x] ++- Join Inner + :- SubqueryAlias WITH_NULLS + : +- Aggregate [kll_merge_agg_bigint(sketch_col#x, None, 0, 0) AS agg_with_nulls#x] + : +- SubqueryAlias sketches_with_nulls + : +- Union false, false + : :- Union false, false + : : :- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS sketch_col#x] + : : : +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + : : : +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + : : +- Project [cast(null as binary) AS sketch_col#x] + : : +- OneRowRelation + : +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS sketch_col#x] + : +- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 + : +- Relation spark_catalog.default.t_byte_1_5_through_7_11[col1#x,col2#x] parquet + +- SubqueryAlias WITHOUT_NULLS + +- Aggregate [kll_merge_agg_bigint(sketch_col#x, None, 0, 0) AS agg_without_nulls#x] + +- SubqueryAlias sketches_without_nulls + +- Union false, false + :- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS sketch_col#x] + : +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 + : +- Relation spark_catalog.default.t_long_1_5_through_7_11[col1#xL,col2#xL] parquet + +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 + +- Relation spark_catalog.default.t_byte_1_5_through_7_11[col1#x,col2#x] parquet + + -- !query SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match, @@ -1054,6 +1310,235 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT kll_merge_agg_bigint(sketch_col) AS wrong_type_merge +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 +) float_sketches +-- !query analysis +Aggregate [kll_merge_agg_bigint(sketch_col#x, None, 0, 0) AS wrong_type_merge#x] ++- SubqueryAlias float_sketches + +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS sketch_col#x] + +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 + +- Relation spark_catalog.default.t_float_1_5_through_7_11[col1#x,col2#x] parquet + + +-- !query +SELECT kll_merge_agg_bigint(col1) AS merge_wrong_type +FROM t_long_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_merge_agg_bigint(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_merge_agg_bigint(col1)" + } ] +} + + +-- !query +SELECT kll_merge_agg_float(col1) AS merge_wrong_type +FROM t_float_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_merge_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "kll_merge_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_merge_agg_double(col1) AS merge_wrong_type +FROM t_double_1_5_through_7_11 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_merge_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_merge_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_merge_agg_bigint(sketch_col) AS invalid_merge +FROM ( + SELECT CAST('not_a_sketch' AS BINARY) AS sketch_col +) invalid_data +-- !query analysis +Aggregate [kll_merge_agg_bigint(sketch_col#x, None, 0, 0) AS invalid_merge#x] ++- SubqueryAlias invalid_data + +- Project [cast(not_a_sketch as binary) AS sketch_col#x] + +- OneRowRelation + + +-- !query +SELECT kll_merge_agg_float(sketch_col) AS invalid_merge +FROM ( + SELECT X'deadbeef' AS sketch_col +) invalid_data +-- !query analysis +Aggregate [kll_merge_agg_float(sketch_col#x, None, 0, 0) AS invalid_merge#x] ++- SubqueryAlias invalid_data + +- Project [0xDEADBEEF AS sketch_col#x] + +- OneRowRelation + + +-- !query +SELECT kll_merge_agg_double(sketch_col) AS invalid_merge +FROM ( + SELECT X'cafebabe' AS sketch_col +) invalid_data +-- !query analysis +Aggregate [kll_merge_agg_double(sketch_col#x, None, 0, 0) AS invalid_merge#x] ++- SubqueryAlias invalid_data + +- Project [0xCAFEBABE AS sketch_col#x] + +- OneRowRelation + + +-- !query +SELECT kll_merge_agg_bigint(sketch_col, 7) AS k_too_small +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 +) sketches +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_merge_agg_bigint`, the k parameter must be between 8 and 65535 (inclusive), but got 7. SQLSTATE: 22003", + "sqlExpr" : "\"kll_merge_agg_bigint(sketch_col, 7)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "kll_merge_agg_bigint(sketch_col, 7)" + } ] +} + + +-- !query +SELECT kll_merge_agg_float(sketch_col, 65536) AS k_too_large +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 +) sketches +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_merge_agg_float`, the k parameter must be between 8 and 65535 (inclusive), but got 65536. SQLSTATE: 22003", + "sqlExpr" : "\"kll_merge_agg_float(sketch_col, 65536)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "kll_merge_agg_float(sketch_col, 65536)" + } ] +} + + +-- !query +SELECT kll_merge_agg_double(sketch_col, CAST(NULL AS INT)) AS k_is_null +FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 +) sketches +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "k", + "sqlExpr" : "\"kll_merge_agg_double(sketch_col, CAST(NULL AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "kll_merge_agg_double(sketch_col, CAST(NULL AS INT))" + } ] +} + + +-- !query +SELECT kll_merge_agg_bigint(sketch_col, CAST(RAND() * 100 AS INT) + 200) AS k_non_constant +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 +) sketches +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "(CAST((rand() * CAST(100 AS DOUBLE)) AS INT) + 200)", + "inputName" : "k", + "inputType" : "int", + "sqlExpr" : "\"kll_merge_agg_bigint(sketch_col, (CAST((rand() * 100) AS INT) + 200))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "kll_merge_agg_bigint(sketch_col, CAST(RAND() * 100 AS INT) + 200)" + } ] +} + + -- !query SELECT kll_sketch_get_n_bigint(X'deadbeef') AS invalid_binary_bigint -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql index fe1b61de037d7..9300754d204a5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql @@ -123,7 +123,7 @@ FROM ( FROM t_float_1_5_through_7_11 ); --- Merging sketches and converting them to strings +-- Merging sketches and converting them to strings (scalar merge functions) SELECT split( kll_sketch_to_string_bigint( @@ -160,6 +160,143 @@ SELECT )[1] AS result FROM t_byte_1_5_through_7_11; +-- Tests for KllMergeAgg* aggregate functions +-- These functions merge multiple binary sketch representations + +-- Test GROUP BY with kll_merge_agg_bigint and HAVING clause +SELECT + parity, + kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) AS total_count +FROM ( + SELECT + col1 % 2 AS parity, + kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + GROUP BY col1 % 2 +) grouped_sketches +GROUP BY parity +HAVING kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) > 3; + +-- Test empty aggregation: zero rows input for kll_merge_agg_bigint +SELECT kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + WHERE col1 > 1000 +) empty_sketches; + +-- Test empty aggregation: zero rows input for kll_merge_agg_float +SELECT kll_sketch_get_n_float(kll_merge_agg_float(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 + WHERE col1 > 1000.0 +) empty_sketches; + +-- Test empty aggregation: zero rows input for kll_merge_agg_double +SELECT kll_sketch_get_n_double(kll_merge_agg_double(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 + WHERE col1 > 1000.0 +) empty_sketches; + +-- Test kll_merge_agg_bigint: merge bigint sketches from multiple rows +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_merge_agg_bigint(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_short_1_5_through_7_11 + ) sketches +); + +-- Test kll_merge_agg_float: merge float sketches from multiple rows +-- Merging col1 (1-7) and col2 (5-11) gives combined data with median ~5.5 +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 5.5) < 1.0 AS median_close_to_5_5, + abs(kll_sketch_get_rank_float(agg, 5.0) - 0.35) < 0.15 AS rank5_close_to_0_35 +FROM ( + SELECT kll_merge_agg_float(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_float(col2) AS sketch_col + FROM t_float_1_5_through_7_11 + ) sketches +); + +-- Test kll_merge_agg_double: merge double sketches from multiple rows +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 6.0) < 1.0 AS median_close_to_6, + abs(kll_sketch_get_rank_double(agg, 5.0) - 0.35) < 0.15 AS rank5_close_to_0_35 +FROM ( + SELECT kll_merge_agg_double(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_double(col2) AS sketch_col + FROM t_float_1_5_through_7_11 + ) sketches +); + +-- Test kll_merge_agg_bigint with custom k parameter +SELECT LENGTH(kll_sketch_to_string_bigint(kll_merge_agg_bigint(sketch_col, 400))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_bigint(col1, 400) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col2, 400) AS sketch_col + FROM t_byte_1_5_through_7_11 +) sketches; + +-- Test kll_merge_agg_float with custom k parameter +SELECT LENGTH(kll_sketch_to_string_float(kll_merge_agg_float(sketch_col, 300))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_float(col1, 300) AS sketch_col + FROM t_float_1_5_through_7_11 +) sketches; + +-- Test kll_merge_agg_double with custom k parameter +SELECT LENGTH(kll_sketch_to_string_double(kll_merge_agg_double(sketch_col, 500))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_double(col1, 500) AS sketch_col + FROM t_double_1_5_through_7_11 +) sketches; + +-- Test that kll_merge_agg functions ignore NULL sketch values +SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match +FROM ( + SELECT kll_merge_agg_bigint(sketch_col) AS agg_with_nulls + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT CAST(NULL AS BINARY) AS sketch_col + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_byte_1_5_through_7_11 + ) sketches_with_nulls +) WITH_NULLS, +( + SELECT kll_merge_agg_bigint(sketch_col) AS agg_without_nulls + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_byte_1_5_through_7_11 + ) sketches_without_nulls +) WITHOUT_NULLS; + -- Tests verifying that NULL input values are ignored by aggregate functions -- Test BIGINT aggregate ignores NULL values @@ -405,6 +542,73 @@ FROM t_double_1_5_through_7_11; SELECT kll_sketch_agg_bigint(col1, '100') AS k_wrong_type FROM t_long_1_5_through_7_11; +-- Negative tests for kll_merge_agg functions + +-- Test wrong sketch type: float sketch passed to kll_merge_agg_bigint (should fail) +SELECT kll_merge_agg_bigint(sketch_col) AS wrong_type_merge +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 +) float_sketches; + +-- Type mismatch: kll_merge_agg_bigint does not accept integer columns (needs binary) +SELECT kll_merge_agg_bigint(col1) AS merge_wrong_type +FROM t_long_1_5_through_7_11; + +-- Type mismatch: kll_merge_agg_float does not accept float columns (needs binary) +SELECT kll_merge_agg_float(col1) AS merge_wrong_type +FROM t_float_1_5_through_7_11; + +-- Type mismatch: kll_merge_agg_double does not accept double columns (needs binary) +SELECT kll_merge_agg_double(col1) AS merge_wrong_type +FROM t_double_1_5_through_7_11; + +-- Invalid binary data for kll_merge_agg_bigint +SELECT kll_merge_agg_bigint(sketch_col) AS invalid_merge +FROM ( + SELECT CAST('not_a_sketch' AS BINARY) AS sketch_col +) invalid_data; + +-- Invalid binary data for kll_merge_agg_float +SELECT kll_merge_agg_float(sketch_col) AS invalid_merge +FROM ( + SELECT X'deadbeef' AS sketch_col +) invalid_data; + +-- Invalid binary data for kll_merge_agg_double +SELECT kll_merge_agg_double(sketch_col) AS invalid_merge +FROM ( + SELECT X'cafebabe' AS sketch_col +) invalid_data; + +-- k parameter too small for kll_merge_agg_bigint +SELECT kll_merge_agg_bigint(sketch_col, 7) AS k_too_small +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 +) sketches; + +-- k parameter too large for kll_merge_agg_float +SELECT kll_merge_agg_float(sketch_col, 65536) AS k_too_large +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 +) sketches; + +-- k parameter is NULL for kll_merge_agg_double +SELECT kll_merge_agg_double(sketch_col, CAST(NULL AS INT)) AS k_is_null +FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 +) sketches; + +-- k parameter is not foldable for kll_merge_agg_bigint (using a non-constant expression) +SELECT kll_merge_agg_bigint(sketch_col, CAST(RAND() * 100 AS INT) + 200) AS k_non_constant +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 +) sketches; + -- Negative tests for kll_sketch_get_n functions -- Invalid binary data SELECT kll_sketch_get_n_bigint(X'deadbeef') AS invalid_binary_bigint; diff --git a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out index fef44781e17cc..0b852ad24199d 100644 --- a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out @@ -321,6 +321,194 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT + parity, + kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) AS total_count +FROM ( + SELECT + col1 % 2 AS parity, + kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + GROUP BY col1 % 2 +) grouped_sketches +GROUP BY parity +HAVING kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) > 3 +-- !query schema +struct +-- !query output +1 4 + + +-- !query +SELECT kll_sketch_get_n_bigint(kll_merge_agg_bigint(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + WHERE col1 > 1000 +) empty_sketches +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT kll_sketch_get_n_float(kll_merge_agg_float(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 + WHERE col1 > 1000.0 +) empty_sketches +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT kll_sketch_get_n_double(kll_merge_agg_double(sketch_col)) AS empty_merge_n +FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 + WHERE col1 > 1000.0 +) empty_sketches +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, + abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 +FROM ( + SELECT kll_merge_agg_bigint(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_int_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_short_1_5_through_7_11 + ) sketches +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_float(agg, 0.5) - 5.5) < 1.0 AS median_close_to_5_5, + abs(kll_sketch_get_rank_float(agg, 5.0) - 0.35) < 0.15 AS rank5_close_to_0_35 +FROM ( + SELECT kll_merge_agg_float(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_float(col2) AS sketch_col + FROM t_float_1_5_through_7_11 + ) sketches +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, + abs(kll_sketch_get_quantile_double(agg, 0.5) - 6.0) < 1.0 AS median_close_to_6, + abs(kll_sketch_get_rank_double(agg, 5.0) - 0.35) < 0.15 AS rank5_close_to_0_35 +FROM ( + SELECT kll_merge_agg_double(sketch_col) AS agg + FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_double(col2) AS sketch_col + FROM t_float_1_5_through_7_11 + ) sketches +) +-- !query schema +struct +-- !query output +true true true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_bigint(kll_merge_agg_bigint(sketch_col, 400))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_bigint(col1, 400) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col2, 400) AS sketch_col + FROM t_byte_1_5_through_7_11 +) sketches +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_float(kll_merge_agg_float(sketch_col, 300))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_float(col1, 300) AS sketch_col + FROM t_float_1_5_through_7_11 +) sketches +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT LENGTH(kll_sketch_to_string_double(kll_merge_agg_double(sketch_col, 500))) > 0 AS merged_with_k +FROM ( + SELECT kll_sketch_agg_double(col1, 500) AS sketch_col + FROM t_double_1_5_through_7_11 +) sketches +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - + kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match +FROM ( + SELECT kll_merge_agg_bigint(sketch_col) AS agg_with_nulls + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT CAST(NULL AS BINARY) AS sketch_col + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_byte_1_5_through_7_11 + ) sketches_with_nulls +) WITH_NULLS, +( + SELECT kll_merge_agg_bigint(sketch_col) AS agg_without_nulls + FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 + UNION ALL + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_byte_1_5_through_7_11 + ) sketches_without_nulls +) WITHOUT_NULLS +-- !query schema +struct +-- !query output +true + + -- !query SELECT abs(kll_sketch_get_quantile_bigint(agg_with_nulls, 0.5) - kll_sketch_get_quantile_bigint(agg_without_nulls, 0.5)) < 1 AS medians_match, @@ -1085,6 +1273,272 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT kll_merge_agg_bigint(sketch_col) AS wrong_type_merge +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 +) float_sketches +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22000", + "messageParameters" : { + "function" : "`kll_merge_agg_bigint`" + } +} + + +-- !query +SELECT kll_merge_agg_bigint(col1) AS merge_wrong_type +FROM t_long_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_merge_agg_bigint(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_merge_agg_bigint(col1)" + } ] +} + + +-- !query +SELECT kll_merge_agg_float(col1) AS merge_wrong_type +FROM t_float_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_merge_agg_float(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "kll_merge_agg_float(col1)" + } ] +} + + +-- !query +SELECT kll_merge_agg_double(col1) AS merge_wrong_type +FROM t_double_1_5_through_7_11 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"kll_merge_agg_double(col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "kll_merge_agg_double(col1)" + } ] +} + + +-- !query +SELECT kll_merge_agg_bigint(sketch_col) AS invalid_merge +FROM ( + SELECT CAST('not_a_sketch' AS BINARY) AS sketch_col +) invalid_data +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22000", + "messageParameters" : { + "function" : "`kll_merge_agg_bigint`" + } +} + + +-- !query +SELECT kll_merge_agg_float(sketch_col) AS invalid_merge +FROM ( + SELECT X'deadbeef' AS sketch_col +) invalid_data +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22000", + "messageParameters" : { + "function" : "`kll_merge_agg_float`" + } +} + + +-- !query +SELECT kll_merge_agg_double(sketch_col) AS invalid_merge +FROM ( + SELECT X'cafebabe' AS sketch_col +) invalid_data +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "KLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22000", + "messageParameters" : { + "function" : "`kll_merge_agg_double`" + } +} + + +-- !query +SELECT kll_merge_agg_bigint(sketch_col, 7) AS k_too_small +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 +) sketches +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_merge_agg_bigint`, the k parameter must be between 8 and 65535 (inclusive), but got 7. SQLSTATE: 22003", + "sqlExpr" : "\"kll_merge_agg_bigint(sketch_col, 7)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "kll_merge_agg_bigint(sketch_col, 7)" + } ] +} + + +-- !query +SELECT kll_merge_agg_float(sketch_col, 65536) AS k_too_large +FROM ( + SELECT kll_sketch_agg_float(col1) AS sketch_col + FROM t_float_1_5_through_7_11 +) sketches +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "[KLL_SKETCH_K_OUT_OF_RANGE] For function `kll_merge_agg_float`, the k parameter must be between 8 and 65535 (inclusive), but got 65536. SQLSTATE: 22003", + "sqlExpr" : "\"kll_merge_agg_float(sketch_col, 65536)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "kll_merge_agg_float(sketch_col, 65536)" + } ] +} + + +-- !query +SELECT kll_merge_agg_double(sketch_col, CAST(NULL AS INT)) AS k_is_null +FROM ( + SELECT kll_sketch_agg_double(col1) AS sketch_col + FROM t_double_1_5_through_7_11 +) sketches +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "k", + "sqlExpr" : "\"kll_merge_agg_double(sketch_col, CAST(NULL AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "kll_merge_agg_double(sketch_col, CAST(NULL AS INT))" + } ] +} + + +-- !query +SELECT kll_merge_agg_bigint(sketch_col, CAST(RAND() * 100 AS INT) + 200) AS k_non_constant +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS sketch_col + FROM t_long_1_5_through_7_11 +) sketches +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "(CAST((rand() * CAST(100 AS DOUBLE)) AS INT) + 200)", + "inputName" : "k", + "inputType" : "int", + "sqlExpr" : "\"kll_merge_agg_bigint(sketch_col, (CAST((rand() * 100) AS INT) + 200))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "kll_merge_agg_bigint(sketch_col, CAST(RAND() * 100 AS INT) + 200)" + } ] +} + + -- !query SELECT kll_sketch_get_n_bigint(X'deadbeef') AS invalid_binary_bigint -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 0dfd37ebeae00..bfe15b33768b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -3467,6 +3467,110 @@ class DataFrameAggregateSuite extends QueryTest // Should only count non-null values assert(n == 3L) } + + test("kll_merge_agg_bigint basic functionality") { + // Create two separate sketches + val df1 = Seq(1, 2, 3).toDF("value") + val df2 = Seq(4, 5, 6).toDF("value") + + val sketch1 = df1.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + val sketch2 = df2.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + // Union the sketches and merge them + val merged = sketch1.union(sketch2) + .agg(kll_merge_agg_bigint($"sketch").alias("merged_sketch")) + + // Verify the merged sketch contains all values + val n = merged.select(kll_sketch_get_n_bigint($"merged_sketch")).collect()(0)(0) + assert(n == 6L) + + // Test with explicit k parameter + val mergedWithK = sketch1.union(sketch2) + .agg(kll_merge_agg_bigint($"sketch", 400).alias("merged_sketch")) + assert(mergedWithK.collect()(0)(0) != null) + + // Test with column name + val mergedWithName = sketch1.union(sketch2) + .agg(kll_merge_agg_bigint("sketch").alias("merged_sketch")) + val n2 = mergedWithName.select(kll_sketch_get_n_bigint($"merged_sketch")).collect()(0)(0) + assert(n2 == 6L) + } + + test("kll_merge_agg_float basic functionality") { + // Create two separate sketches + val df1 = Seq(1.0f, 2.0f, 3.0f).toDF("value") + val df2 = Seq(4.0f, 5.0f, 6.0f).toDF("value") + + val sketch1 = df1.agg(kll_sketch_agg_float($"value").alias("sketch")) + val sketch2 = df2.agg(kll_sketch_agg_float($"value").alias("sketch")) + + // Union the sketches and merge them + val merged = sketch1.union(sketch2) + .agg(kll_merge_agg_float($"sketch").alias("merged_sketch")) + + // Verify the merged sketch contains all values + val n = merged.select(kll_sketch_get_n_float($"merged_sketch")).collect()(0)(0) + assert(n == 6L) + + // Test with explicit k parameter + val mergedWithK = sketch1.union(sketch2) + .agg(kll_merge_agg_float($"sketch", 300).alias("merged_sketch")) + assert(mergedWithK.collect()(0)(0) != null) + } + + test("kll_merge_agg_double basic functionality") { + // Create two separate sketches + val df1 = Seq(1.0, 2.0, 3.0).toDF("value") + val df2 = Seq(4.0, 5.0, 6.0).toDF("value") + + val sketch1 = df1.agg(kll_sketch_agg_double($"value").alias("sketch")) + val sketch2 = df2.agg(kll_sketch_agg_double($"value").alias("sketch")) + + // Union the sketches and merge them + val merged = sketch1.union(sketch2) + .agg(kll_merge_agg_double($"sketch").alias("merged_sketch")) + + // Verify the merged sketch contains all values + val n = merged.select(kll_sketch_get_n_double($"merged_sketch")).collect()(0)(0) + assert(n == 6L) + + // Test quantile on merged sketch + val quantile = merged.select( + kll_sketch_get_quantile_double($"merged_sketch", lit(0.5)) + ).collect()(0)(0) + assert(quantile != null) + } + + test("kll_merge_agg with different k values") { + // Create sketches with different k values + val df1 = Seq(1, 2, 3).toDF("value") + val df2 = Seq(4, 5, 6).toDF("value") + + val sketch1 = df1.agg(kll_sketch_agg_bigint($"value", 200).alias("sketch")) + val sketch2 = df2.agg(kll_sketch_agg_bigint($"value", 400).alias("sketch")) + + // Merge sketches with different k values (should adopt from first sketch) + val merged = sketch1.union(sketch2) + .agg(kll_merge_agg_bigint($"sketch").alias("merged_sketch")) + + val n = merged.select(kll_sketch_get_n_bigint($"merged_sketch")).collect()(0)(0) + assert(n == 6L) + } + + test("kll_merge_agg with null values") { + val df1 = Seq(1, 2, 3).toDF("value") + val dfNull = Seq(Some(4), None, Some(6)).toDF("value") + + val sketch1 = df1.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + val sketchNull = dfNull.agg(kll_sketch_agg_bigint($"value").alias("sketch")) + + // Merge sketch with null - null should be ignored + val merged = sketch1.union(sketchNull) + .agg(kll_merge_agg_bigint($"sketch").alias("merged_sketch")) + + val n = merged.select(kll_sketch_get_n_bigint($"merged_sketch")).collect()(0)(0) + assert(n == 5L) + } } case class B(c: Option[Double]) From 48fee2ed09335449fd7ef4a6bf886c7e99be72a1 Mon Sep 17 00:00:00 2001 From: Garland Zhang Date: Thu, 15 Jan 2026 14:25:13 -0400 Subject: [PATCH 353/400] [SPARK-54886] Lazily create Spark Connect reuse session MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This is a follow up to https://github.com/apache/spark/pull/52895 Make the Spark Connect base session created to be lazy. ### Why are the changes needed? Prevoiusly the session is created while initializing the SparkConnectService and this happens in SparkContext initialization. This PR makes it lazy so that only when the SparkContext is fully initialized the session can be created ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #53812 from garlandz-db/SPARK-54886_lazy. Authored-by: Garland Zhang Signed-off-by: Herman van Hövell (cherry picked from commit f310f4fcc95580a6824bc7d22b76006f79b8804a) Signed-off-by: Herman van Hövell --- .../connect/service/SparkConnectService.scala | 5 ++++- .../service/SparkConnectSessionManager.scala | 20 ++++++++++++++----- .../planner/SparkConnectServiceSuite.scala | 2 +- .../ArtifactStatusesHandlerSuite.scala | 2 +- .../SparkConnectCloneSessionSuite.scala | 2 +- .../SparkConnectSessionManagerSuite.scala | 8 ++++---- 6 files changed, 26 insertions(+), 13 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 4641bc0a11067..00b93c19b2c73 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -38,6 +38,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.HOST import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerEvent} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.connect.config.Connect.{getAuthenticateToken, CONNECT_GRPC_BINDING_ADDRESS, CONNECT_GRPC_BINDING_PORT, CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT, CONNECT_GRPC_MAX_INBOUND_MESSAGE_SIZE, CONNECT_GRPC_PORT_MAX_RETRIES} import org.apache.spark.sql.connect.execution.ConnectProgressExecutionListener import org.apache.spark.sql.connect.ui.{SparkConnectServerAppStatusStore, SparkConnectServerListener, SparkConnectServerTab} @@ -436,7 +438,8 @@ object SparkConnectService extends Logging { return } - sessionManager.initializeBaseSession(sc) + sessionManager.initializeBaseSession(() => + SparkSession.builder().sparkContext(sc).getOrCreate().newSession()) startGRPCService() createListenerAndUI(sc) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala index 6c468ba46cc03..d3ddf592e9e7d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import com.google.common.cache.CacheBuilder -import org.apache.spark.{SparkContext, SparkEnv, SparkSQLException} +import org.apache.spark.{SparkEnv, SparkSQLException} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{INTERVAL, SESSION_HOLD_INFO} import org.apache.spark.sql.classic.SparkSession @@ -39,8 +39,11 @@ import org.apache.spark.util.ThreadUtils */ class SparkConnectSessionManager extends Logging { + // Used to lazily initialize the base session + @volatile private var baseSessionCreator: Option[() => SparkSession] = None + // Base SparkSession created from the SparkContext, used to create new isolated sessions - @volatile private var baseSession: Option[SparkSession] = None + @volatile private var _baseSession: Option[SparkSession] = None private val sessionStore: ConcurrentMap[SessionKey, SessionHolder] = new ConcurrentHashMap[SessionKey, SessionHolder]() @@ -51,13 +54,20 @@ class SparkConnectSessionManager extends Logging { .maximumSize(SparkEnv.get.conf.get(CONNECT_SESSION_MANAGER_CLOSED_SESSIONS_TOMBSTONES_SIZE)) .build[SessionKey, SessionHolderInfo]() + private def baseSession: Option[SparkSession] = { + if (_baseSession.isEmpty && baseSessionCreator.isDefined) { + _baseSession = Some(baseSessionCreator.get()) + } + _baseSession + } + /** * Initialize the base SparkSession from the provided SparkContext. This should be called once * during SparkConnectService startup. */ - def initializeBaseSession(sc: SparkContext): Unit = { - if (baseSession.isEmpty) { - baseSession = Some(SparkSession.builder().sparkContext(sc).getOrCreate().newSession()) + def initializeBaseSession(createSession: () => SparkSession): Unit = { + if (baseSessionCreator.isEmpty) { + baseSessionCreator = Some(createSession) } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala index 6df50b6588c24..0e5488e312220 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala @@ -68,7 +68,7 @@ class SparkConnectServiceSuite override def beforeEach(): Unit = { super.beforeEach() SparkConnectService.sessionManager.invalidateAllSessions() - SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) + SparkConnectService.sessionManager.initializeBaseSession(() => spark.newSession()) } private def sparkSessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala index caa71c644e6ac..275808942d37d 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala @@ -45,7 +45,7 @@ class ArtifactStatusesHandlerSuite extends SharedSparkSession with ResourceHelpe override def beforeEach(): Unit = { super.beforeEach() SparkConnectService.sessionManager.invalidateAllSessions() - SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) + SparkConnectService.sessionManager.initializeBaseSession(() => spark.newSession()) } def getStatuses(names: Seq[String], exist: Set[String]): ArtifactStatusesResponse = { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala index 42541b8c5f002..044103a3e4f13 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectCloneSessionSuite.scala @@ -29,7 +29,7 @@ class SparkConnectCloneSessionSuite extends SharedSparkSession with BeforeAndAft override def beforeEach(): Unit = { super.beforeEach() SparkConnectService.sessionManager.invalidateAllSessions() - SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) + SparkConnectService.sessionManager.initializeBaseSession(() => spark.newSession()) } test("clone session with invalid target session ID format") { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala index 04d16a910746b..4029e4775f44b 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala @@ -33,7 +33,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA override def beforeEach(): Unit = { super.beforeEach() SparkConnectService.sessionManager.invalidateAllSessions() - SparkConnectService.sessionManager.initializeBaseSession(spark.sparkContext) + SparkConnectService.sessionManager.initializeBaseSession(() => spark.newSession()) } test("sessionId needs to be an UUID") { @@ -179,7 +179,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val sessionManager = new SparkConnectSessionManager() // Initialize the base session with the test SparkContext - sessionManager.initializeBaseSession(spark.sparkContext) + sessionManager.initializeBaseSession(() => spark.newSession()) // Clear the default and active sessions to simulate the scenario where // SparkSession.active or SparkSession.getDefaultSession would fail @@ -203,13 +203,13 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val sessionManager = new SparkConnectSessionManager() // Initialize the base session multiple times - sessionManager.initializeBaseSession(spark.sparkContext) + sessionManager.initializeBaseSession(() => spark.newSession()) val key1 = SessionKey("user1", UUID.randomUUID().toString) val sessionHolder1 = sessionManager.getOrCreateIsolatedSession(key1, None) val baseSessionUUID1 = sessionHolder1.session.sessionUUID // Initialize again - should not change the base session - sessionManager.initializeBaseSession(spark.sparkContext) + sessionManager.initializeBaseSession(() => spark.newSession()) val key2 = SessionKey("user2", UUID.randomUUID().toString) val sessionHolder2 = sessionManager.getOrCreateIsolatedSession(key2, None) From 86c6548697ed1a93bef0139b17d2ffe4edcbfdc9 Mon Sep 17 00:00:00 2001 From: ericm-db Date: Fri, 16 Jan 2026 13:49:01 +0900 Subject: [PATCH 354/400] [SPARK-55015][SS][SQL] Fix decodeRemainingKey numFields calculation in PrefixKeyScanStateEncoder ### What changes were proposed in this pull request? Fix bug in RocksDBStateEncoder.decodeRemainingKey where it incorrectly used numColsPrefixKey instead of (keySchema.length - numColsPrefixKey) for the PrefixKeyScanStateEncoderSpec case. The remaining key should contain the non-prefix columns, not the prefix columns. ### Why are the changes needed? On the decode path, we would hit an exception ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added test to verify decodeRemainingKey correctly decodes with the proper number of fields. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53775 from ericm-db/fix-decode-remaining-key. Authored-by: ericm-db Signed-off-by: Jungtaek Lim (cherry picked from commit af058d59f3008163fd8c72ce80fa5ca24706165a) Signed-off-by: Jungtaek Lim --- .../streaming/state/RocksDBStateEncoder.scala | 2 +- .../streaming/state/RocksDBSuite.scala | 124 ++++++++++++++++++ 2 files changed, 125 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index f49c79f96b9ce..f6fe4dbea576c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -628,7 +628,7 @@ class UnsafeRowDataEncoder( override def decodeRemainingKey(bytes: Array[Byte]): UnsafeRow = { keyStateEncoderSpec match { case PrefixKeyScanStateEncoderSpec(_, numColsPrefixKey) => - decodeToUnsafeRow(bytes, numFields = numColsPrefixKey) + decodeToUnsafeRow(bytes, numFields = keySchema.length - numColsPrefixKey) case RangeKeyScanStateEncoderSpec(_, orderingOrdinals) => decodeToUnsafeRow(bytes, keySchema.length - orderingOrdinals.length) case _ => throw unsupportedOperationForKeyStateEncoder("decodeRemainingKey") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 6c22436c29a01..21ce069f3b3b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -584,6 +584,130 @@ class RocksDBStateEncoderSuite extends SparkFunSuite { assert(decodedValue.getBoolean(2) === true) } } + + test("verify PrefixKeyScanStateEncoder full encode/decode cycle with multi-key session window") { + // Simulate session window state with multiple grouping keys + // Key schema: [userId, deviceId, sessionStartTime] - mimics session window with 2 grouping keys + val keySchema = StructType(Seq( + StructField("userId", IntegerType), + StructField("deviceId", StringType), + StructField("sessionStartTime", LongType) + )) + val valueSchema = StructType(Seq( + StructField("count", LongType) + )) + + // Session window uses first N columns as prefix (the grouping keys) + val numColsPrefixKey = 2 + val prefixKeySpec = PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) + val dataEncoder = new UnsafeRowDataEncoder(prefixKeySpec, valueSchema) + val keyEncoder = new PrefixKeyScanStateEncoder( + dataEncoder, keySchema, numColsPrefixKey, useColumnFamilies = false) + + // Create a full key row + val keyProj = UnsafeProjection.create(keySchema) + val fullKey = keyProj.apply(InternalRow(123, UTF8String.fromString("device1"), 1000000L)) + + // Encode the full key (this is what happens when putting to state store) + val encodedKey = keyEncoder.encodeKey(fullKey) + + // Decode the key (this is what happens during prefix scan) + val decodedKey = keyEncoder.decodeKey(encodedKey) + + // Verify the decoded key matches the original + assert(decodedKey.numFields === 3, + s"Expected 3 fields in decoded key, but got ${decodedKey.numFields}") + assert(decodedKey.getInt(0) === 123, "userId not preserved") + assert(decodedKey.getString(1) === "device1", "deviceId not preserved") + assert(decodedKey.getLong(2) === 1000000L, "sessionStartTime not preserved") + } + + test("verify decodeRemainingKey correctly decodes with fix") { + // This test verifies the fix prevents garbage data reads + val keySchema = StructType(Seq( + StructField("k1", IntegerType), + StructField("k2", StringType), + StructField("k3", LongType) + )) + val valueSchema = StructType(Seq( + StructField("v1", IntegerType) + )) + + val prefixKeySpec = PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey = 2) + val encoder = new UnsafeRowDataEncoder(prefixKeySpec, valueSchema) + + // Create and encode a remaining key with just the last column (k3) + val remainingKeySchema = StructType(Seq(StructField("k3", LongType))) + val remainingKeyProj = UnsafeProjection.create(remainingKeySchema) + val remainingKeyRow = remainingKeyProj.apply(InternalRow(999999L)) + val encodedRemainingKey = encoder.encodeRemainingKey(remainingKeyRow) + + // Decode the remaining key + val decodedRemainingKey = encoder.decodeRemainingKey(encodedRemainingKey) + + // With the FIX: numFields should be keySchema.length - numColsPrefixKey = 3 - 2 = 1 + assert(decodedRemainingKey.numFields === 1, + s"Expected 1 field but got ${decodedRemainingKey.numFields}") + + // Field 0 should read correctly + assert(decodedRemainingKey.getLong(0) === 999999L, + "Field 0 value incorrect") + + // Trying to read field 1 should throw exception (doesn't exist) + intercept[AssertionError] { + decodedRemainingKey.getLong(1) + } + } + + test("verify AvroStateEncoder decodeRemainingKey with PrefixKeyScanStateEncoder") { + // This test verifies that AvroStateEncoder correctly decodes remaining keys + // AvroStateEncoder uses remainingKeySchema = keySchema.drop(numColsPrefixKey) + // which is the correct calculation (unlike the bug in UnsafeRowDataEncoder) + val keySchema = StructType(Seq( + StructField("k1", IntegerType), + StructField("k2", StringType), + StructField("k3", LongType) + )) + val valueSchema = StructType(Seq( + StructField("v1", IntegerType) + )) + + // Create test state schema provider + val testProvider = new TestStateSchemaProvider() + testProvider.captureSchema( + StateStore.DEFAULT_COL_FAMILY_NAME, + keySchema, + valueSchema, + keySchemaId = 0, + valueSchemaId = 0 + ) + + val prefixKeySpec = PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey = 2) + val encoder = new AvroStateEncoder(prefixKeySpec, valueSchema, Some(testProvider), + StateStore.DEFAULT_COL_FAMILY_NAME) + + // Create and encode a remaining key with just the last column (k3) + val remainingKeySchema = StructType(Seq(StructField("k3", LongType))) + val remainingKeyProj = UnsafeProjection.create(remainingKeySchema) + val remainingKeyRow = remainingKeyProj.apply(InternalRow(999999L)) + val encodedRemainingKey = encoder.encodeRemainingKey(remainingKeyRow) + + // Decode the remaining key + val decodedRemainingKey = encoder.decodeRemainingKey(encodedRemainingKey) + + // Should have 1 field (keySchema.length - numColsPrefixKey = 3 - 2 = 1) + assert(decodedRemainingKey.numFields === 1, + s"Expected 1 field but got ${decodedRemainingKey.numFields}") + + // Field 0 should read correctly + assert(decodedRemainingKey.getLong(0) === 999999L, + "Field 0 value incorrect") + + // Trying to read field 1 should throw exception (doesn't exist) + intercept[AssertionError] { + decodedRemainingKey.getLong(1) + } + } } @SlowSQLTest From 14489233a35ec07ce4e05092aba7e2e3a4922870 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 16 Jan 2026 10:20:15 -0800 Subject: [PATCH 355/400] [SPARK-43986][SQL][FOLLOWUP] Catch ArrayIndexOutOfBoundsException in Datasketches HLL functions ### What changes were proposed in this pull request? This PR fixes error handling in `HllUnionAgg` to properly catch `ArrayIndexOutOfBoundsException` and convert it to the user-friendly `HLL_INVALID_INPUT_SKETCH_BUFFER` error. When an invalid HLL sketch binary is passed to `HllUnionAgg`, the DataSketches library may throw an `ArrayIndexOutOfBoundsException` during parsing. For example, if the `curMode` ordinal in the HLL preamble is set to an invalid value (e.g., 3 when only 0, 1, 2 are valid), `CurMode.fromOrdinal()` throws this exception. Previously, this exception would propagate to the user as an unhelpful `ArrayIndexOutOfBoundsException`. The fix adds `ArrayIndexOutOfBoundsException` to the existing catch block that already handles `SketchesArgumentException` and `java.lang.Error`, ensuring all invalid sketch buffer errors are converted to the clear `HLL_INVALID_INPUT_SKETCH_BUFFER` error message. ### Why are the changes needed? Without this fix, users see a confusing `ArrayIndexOutOfBoundsException` when passing malformed binary data to HLL union functions. This provides a poor user experience as it doesn't indicate what went wrong or how to fix it. The `HLL_INVALID_INPUT_SKETCH_BUFFER` error clearly indicates that the input binary is not a valid HLL sketch. ### Does this PR introduce _any_ user-facing change? Yes. Users who pass invalid binary data to `hll_union_agg` will now see a clear error message (`HLL_INVALID_INPUT_SKETCH_BUFFER`) instead of an `ArrayIndexOutOfBoundsException`. ### How was this patch tested? Added a new unit test `"HllUnionAgg throws proper error for invalid binary input causing ArrayIndexOutOfBounds"` in `DatasketchesHllSketchSuite` that: 1. Crafts a byte array with a valid HLL preamble structure but an invalid `curMode` ordinal (3 instead of 0, 1, or 2) 2. Verifies that the exception is NOT an `ArrayIndexOutOfBoundsException` 3. Verifies that the error message contains `HLL_INVALID_INPUT_SKETCH_BUFFER` ### Was this patch authored or co-authored using generative AI tooling? Yes, `claude-4.5-opus-high` plus manual review and editing. Closes #53823 from dtenedor/fix-kll-quantiles-errors. Authored-by: Daniel Tenedorio Signed-off-by: Daniel Tenedorio (cherry picked from commit 6939fcd4a313717e0ac425a3bfed7a784908adca) Signed-off-by: Daniel Tenedorio --- .../aggregate/datasketchesAggregates.scala | 3 +- .../expressions/datasketchesExpressions.scala | 9 ++-- .../DatasketchesHllSketchSuite.scala | 45 +++++++++++++++++++ 3 files changed, 53 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala index 8ae9b8fddde71..8fb1bf51319cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala @@ -338,7 +338,8 @@ case class HllUnionAgg( union.update(sketch) Some(union) } catch { - case _: SketchesArgumentException | _: java.lang.Error => + case _: SketchesArgumentException | _: java.lang.Error + | _: ArrayIndexOutOfBoundsException => throw QueryExecutionErrors.hllInvalidInputSketchBuffer(prettyName) } case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala index a4ac0bdbb11d3..1880d71e7d542 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala @@ -56,7 +56,8 @@ case class HllSketchEstimate(child: Expression) try { Math.round(HllSketch.heapify(Memory.wrap(buffer)).getEstimate) } catch { - case _: SketchesArgumentException | _: java.lang.Error => + case _: SketchesArgumentException | _: java.lang.Error + | _: ArrayIndexOutOfBoundsException => throw QueryExecutionErrors.hllInvalidInputSketchBuffer(prettyName) } } @@ -108,13 +109,15 @@ case class HllUnion(first: Expression, second: Expression, third: Expression) val sketch1 = try { HllSketch.heapify(Memory.wrap(value1.asInstanceOf[Array[Byte]])) } catch { - case _: SketchesArgumentException | _: java.lang.Error => + case _: SketchesArgumentException | _: java.lang.Error + | _: ArrayIndexOutOfBoundsException => throw QueryExecutionErrors.hllInvalidInputSketchBuffer(prettyName) } val sketch2 = try { HllSketch.heapify(Memory.wrap(value2.asInstanceOf[Array[Byte]])) } catch { - case _: SketchesArgumentException | _: java.lang.Error => + case _: SketchesArgumentException | _: java.lang.Error + | _: ArrayIndexOutOfBoundsException => throw QueryExecutionErrors.hllInvalidInputSketchBuffer(prettyName) } val allowDifferentLgConfigK = value3.asInstanceOf[Boolean] diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala index 0841702cc5180..0f7f5ca54be01 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala @@ -108,4 +108,49 @@ class DatasketchesHllSketchSuite extends SparkFunSuite { assert(HllSketch.heapify(Memory.wrap(binary3.asInstanceOf[Array[Byte]])).getLgConfigK == 12) } + + test("HllUnionAgg throws proper error for invalid binary input causing ArrayIndexOutOfBounds") { + val aggFunc = new HllUnionAgg(BoundReference(0, BinaryType, nullable = true), true) + val union = aggFunc.createAggregationBuffer() + + // Craft a byte array that passes initial size checks but has an invalid CurMode ordinal. + // HLL preamble layout: + // Byte 0: preInts (preamble size in ints) + // Byte 1: serVer (must be 1) + // Byte 2: famId (must be 7 for HLL) + // Byte 3: lgK (4-21) + // Byte 5: flags + // Byte 7: modeByte - bits 0-1 contain curMode ordinal (0=LIST, 1=SET, 2=HLL) + // + // Setting bits 0-1 of byte 7 to 0b11 (=3) causes CurMode.fromOrdinal(3) to throw + // ArrayIndexOutOfBoundsException since CurMode only has ordinals 0, 1, 2. + // This happens in PreambleUtil.extractCurMode() before other validations run. + val invalidBinary = Array[Byte]( + 2, // byte 0: preInts = 2 (LIST_PREINTS, passes check) + 1, // byte 1: serVer = 1 (valid) + 7, // byte 2: famId = 7 (HLL family) + 12, // byte 3: lgK = 12 (valid range 4-21) + 0, // byte 4: unused + 0, // byte 5: flags = 0 + 0, // byte 6: unused + 3 // byte 7: modeByte with bits 0-1 = 0b11 = 3 (INVALID curMode ordinal!) + ) + + val exception = intercept[Exception] { + aggFunc.update(union, InternalRow(invalidBinary)) + } + + // Verify that ArrayIndexOutOfBoundsException is properly caught and converted + // to the user-friendly HLL_INVALID_INPUT_SKETCH_BUFFER error + assert( + !exception.isInstanceOf[ArrayIndexOutOfBoundsException], + s"ArrayIndexOutOfBoundsException should be caught and converted to " + + s"HLL_INVALID_INPUT_SKETCH_BUFFER error, but got: ${exception.getClass.getName}" + ) + assert( + exception.getMessage.contains("HLL_INVALID_INPUT_SKETCH_BUFFER"), + s"Expected HLL_INVALID_INPUT_SKETCH_BUFFER error, " + + s"but got: ${exception.getClass.getName}: ${exception.getMessage}" + ) + } } From d56f02586289b61a6043ab04ea53f1c781ac6111 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 19 Jan 2026 15:32:45 +0800 Subject: [PATCH 356/400] [SPARK-55070][SQL][CONNECT] Allow hidden column in dataframe column resolution ### What changes were proposed in this pull request? Allow hidden column in dataframe column resolution ### Why are the changes needed? https://github.com/apache/spark/pull/53503 was to fix a regression, but it also introduced another issue: ```py lhs = spark.createDataFrame([(1, 'A'), (2, 'B')], ['ID', 'join_key']) rhs = spark.createDataFrame([(3, 'A'), (4, 'C')], ['ID', 'join_key']) lhs.join(rhs, 'join_key').select(rhs['join_key']) ``` falis after https://github.com/apache/spark/pull/53503 ``` 'join_key[id=3] against [id=6]Project [join_key#39, ID#38L, ID#40L] +- Join Inner, (join_key#39 = join_key#41) :- [id=1]Project [ID#28L AS ID#38L, join_key#29 AS join_key#39] : +- [id=0]LocalRelation [ID#28L, join_key#29] +- [id=3]Project [ID#36L AS ID#40L, join_key#37 AS join_key#41] +- [id=2]LocalRelation [ID#36L, join_key#37] ``` resloving `'join_key[id=3]` against the plan: 1, find the corresponding node `[id=3]Project [ID#36L AS ID#40L, join_key#37 AS join_key#41]`; 2, resolve `'join_key[id=3]` to `join_key#41`; 3, the result was dropped when filtering with `[id=6]Project [join_key#39, ID#38L, ID#40L]` because `join_key#41` is not in the node output; before https://github.com/apache/spark/pull/53503, the steps are: 1, find the corresponding node `[id=3]Project [ID#36L AS ID#40L, join_key#37 AS join_key#41]`; 2, resolve `'join_key[id=3]` to `join_key#41`; 3, the result was dropped; 4, return None, and `resolveExpression` resolves it without the plan id, but incorrectly resolve it to the left key `join_key#39`. ### Does this PR introduce _any_ user-facing change? yes, query fails before this fix ### How was this patch tested? added tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #53832 from zhengruifeng/fix_proj_hidden. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit 76bdd24677e4756bb5bda3d1e40e356fc7c4a941) Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/tests/test_dataframe.py | 7 +++++++ .../sql/catalyst/analysis/ColumnResolutionHelper.scala | 7 ++----- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 75a553b62838e..a726fc85d90a5 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -159,6 +159,13 @@ def test_self_join_IV(self): self.assertTrue(df3.columns, ["id", "value", "id", "value"]) self.assertTrue(df3.count() == 20) + def test_select_join_keys(self): + df1 = self.spark.range(10).withColumn("v1", lit(1)) + df2 = self.spark.range(10).withColumn("v2", lit(2)) + for how in ["inner", "left", "right", "full", "cross"]: + self.assertTrue(df1.join(df2, "id", how).select(df1["id"]).count() >= 0, how) + self.assertTrue(df1.join(df2, "id", how).select(df2["id"]).count() >= 0, how) + def test_lateral_column_alias(self): df1 = self.spark.range(10).select( (col("id") + lit(1)).alias("x"), (col("x") + lit(1)).alias("y") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 870e033642252..1172ecee72236 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -617,11 +617,8 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { // the dataframe column 'df.id' will remain unresolved, and the analyzer // will try to resolve 'id' without plan id later. val filtered = resolved.filter { r => - if (isMetadataAccess) { - r._1.references.subsetOf(AttributeSet(p.output ++ p.metadataOutput)) - } else { - r._1.references.subsetOf(p.outputSet) - } + // A DataFrame column can be resolved as a metadata column, we should keep it. + r._1.references.subsetOf(AttributeSet(p.output ++ p.metadataOutput)) } (filtered, matched) } From ad9766e6a10062e1464ebcc3d12208135a3fa6ee Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 19 Jan 2026 10:12:13 +0100 Subject: [PATCH 357/400] [SPARK-55033][SQL] Fix stringArgs of DSv2 writing commands ### What changes were proposed in this pull request? Replace `val stringArgs` with `def stringArgs` in v2 writing commands, because each call of the method `stringArgs` should return a brand-new `Iterator`, otherwise, subsequent consumers will get an empty result. ### Why are the changes needed? Fix EXPLAIN / UI display. ### Does this PR introduce _any_ user-facing change? Yes, for example, without this fix, I found that some `AppendData` nodes in UI do not contain `Arguments: ...` where they should be, this patch makes them display correctly consistently. ### How was this patch tested? INSERT INTO an Iceberg table and check UI, before vs after. image image ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53794 from pan3793/SPARK-55033. Authored-by: Cheng Pan Signed-off-by: Peter Toth (cherry picked from commit 7a0ea39fd6a902e2b4f37e80ce82ba69d84185d9) Signed-off-by: Peter Toth --- .../apache/spark/sql/catalyst/plans/logical/v2Commands.scala | 4 ++-- .../execution/datasources/v2/WriteToDataSourceV2Exec.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index fab64d771093f..17c5eb1593bfe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -257,7 +257,7 @@ case class ReplaceData( write: Option[Write] = None) extends RowLevelWrite { override val isByName: Boolean = false - override val stringArgs: Iterator[Any] = Iterator(table, query, write) + override def stringArgs: Iterator[Any] = Iterator(table, query, write) override lazy val references: AttributeSet = query.outputSet @@ -339,7 +339,7 @@ case class WriteDelta( write: Option[DeltaWrite] = None) extends RowLevelWrite { override val isByName: Boolean = false - override val stringArgs: Iterator[Any] = Iterator(table, query, write) + override def stringArgs: Iterator[Any] = Iterator(table, query, write) override lazy val references: AttributeSet = query.outputSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 3e4a2f792a1c8..464f0d9658d1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -355,7 +355,7 @@ case class WriteToDataSourceV2Exec( query: SparkPlan, writeMetrics: Seq[CustomMetric]) extends V2TableWriteExec { - override val stringArgs: Iterator[Any] = Iterator(batchWrite, query) + override def stringArgs: Iterator[Any] = Iterator(batchWrite, query) override val customMetrics: Map[String, SQLMetric] = writeMetrics.map { customMetric => customMetric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, customMetric) @@ -375,7 +375,7 @@ trait V2ExistingTableWriteExec extends V2TableWriteExec { def refreshCache: () => Unit def write: Write - override val stringArgs: Iterator[Any] = Iterator(query, write) + override def stringArgs: Iterator[Any] = Iterator(query, write) override val customMetrics: Map[String, SQLMetric] = write.supportedCustomMetrics().map { customMetric => From e5927d8abc4679f1ef8306af644e4f2a8a5c8989 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 20 Jan 2026 18:03:17 +0800 Subject: [PATCH 358/400] [SPARK-55096][PYTHON] Update pandas minimum version in `connect/setup.py` ### What changes were proposed in this pull request? Update pandas minimum version in `connect/setup.py` ### Why are the changes needed? the version had been upgrade to 2.2.0 in spark 4.1, but we forgot to update `connect/setup.py` ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53866 from zhengruifeng/upgrade_pd. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit 5f925fddfff8ff9ad0f2d9816b27ffe07fde2db5) Signed-off-by: Ruifeng Zheng --- python/packaging/connect/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 9a1a4ea812555..fc3cfca739aaa 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -86,7 +86,7 @@ # binary format protocol with the Java version, see ARROW_HOME/format/* for specifications. # Also don't forget to update python/docs/source/getting_started/install.rst, # python/packaging/classic/setup.py, and python/packaging/client/setup.py - _minimum_pandas_version = "2.0.0" + _minimum_pandas_version = "2.2.0" _minimum_numpy_version = "1.21" _minimum_pyarrow_version = "11.0.0" _minimum_grpc_version = "1.76.0" From 7b7a9541701a305e1592238620892095e765d67e Mon Sep 17 00:00:00 2001 From: pranavdev022 Date: Tue, 20 Jan 2026 18:20:14 +0100 Subject: [PATCH 359/400] [SPARK-55097][SQL] Fix re-adding cached artifacts drops blocks silently issue MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? After the introduction of the ref-counting logic for cloning sessions https://github.com/apache/spark/pull/52651, whenever an identical cached artifact (same session, same hash) is re-added, it incorrectly leds to deletion of the existing block. Verified this bug locally using: ``` test("re-adding the same cache artifact should not remove the block") { val blockManager = spark.sparkContext.env.blockManager val remotePath = Paths.get("cache/duplicate_hash") val blockId = CacheId(spark.sessionUUID, "duplicate_hash") try { // First addition withTempPath { path => Files.write(path.toPath, "test".getBytes(StandardCharsets.UTF_8)) artifactManager.addArtifact(remotePath, path.toPath, None) } assert(blockManager.getLocalBytes(blockId).isDefined) blockManager.releaseLock(blockId) // Second addition with same hash - block should still exist withTempPath { path => Files.write(path.toPath, "test".getBytes(StandardCharsets.UTF_8)) artifactManager.addArtifact(remotePath, path.toPath, None) } assert(blockManager.getLocalBytes(blockId).isDefined, "Block should still exist after re-adding the same cache artifact") } finally { blockManager.releaseLock(blockId) blockManager.removeCache(spark.sessionUUID) } } ``` which fails `assert(blockManager.getLocalBytes(blockId).isDefined` check after the second addition with the same hash. Proposed solution: 1. Add early exit check: if `existingBlock.id == blockId`, skip the addition since the block already exists. 2. The fix preserves the intended replacement behavior for cloned sessions (different session UUID, same hash) ### Why are the changes needed? When the same cached artifact is added twice to the same session: - BlockManager.save() detects the block exists and returns without re-adding - hashToCachedIdMap.put() returns the existing RefCountedCacheId - release() decrements ref count to 0 and deletes the block ### Does this PR introduce _any_ user-facing change? No. This is an internal bug fix. Previously, duplicate artifact additions could silently delete cached data causing runtime errors. ### How was this patch tested? Added unit test `cache artifact deduplication and replacement across sessions` in ArtifactManagerSuite that validates: - Duplicate addition in same session is skipped (block survives) - Cloned session can replace inherited artifacts (different CacheId) - Reference counting works correctly across session cleanup ### Was this patch authored or co-authored using generative AI tooling? Co-authored using cursor. Closes #53852 from pranavdev022/fix-artifacts-duplicate-add-issue. Authored-by: pranavdev022 Signed-off-by: Herman van Hövell (cherry picked from commit dfea03626309d04b651fdfa92d58066f29a6b34d) Signed-off-by: Herman van Hövell --- .../spark/sql/artifact/ArtifactManager.scala | 41 ++++++++----- .../sql/artifact/ArtifactManagerSuite.scala | 60 +++++++++++++++---- 2 files changed, 73 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index e11f0f99bf2f6..0055d220a6764 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -183,25 +183,34 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging if (normalizedRemoteRelativePath.startsWith(s"cache${File.separator}")) { val tmpFile = serverLocalStagingPath.toFile Utils.tryWithSafeFinallyAndFailureCallbacks { + val hash = normalizedRemoteRelativePath.toString.stripPrefix(s"cache${File.separator}") val blockManager = session.sparkContext.env.blockManager val blockId = CacheId( sessionUUID = session.sessionUUID, - hash = normalizedRemoteRelativePath.toString.stripPrefix(s"cache${File.separator}")) - val updater = blockManager.TempFileBasedBlockStoreUpdater( - blockId = blockId, - level = StorageLevel.MEMORY_AND_DISK_SER, - classTag = implicitly[ClassTag[Array[Byte]]], - tmpFile = tmpFile, - blockSize = tmpFile.length(), - tellMaster = false) - updater.save() - val oldBlock = hashToCachedIdMap.put(blockId.hash, new RefCountedCacheId(blockId)) - if (oldBlock != null) { - logWarning( - log"Replacing existing cache artifact with hash ${MDC(LogKeys.BLOCK_ID, blockId)} " + - log"in session ${MDC(LogKeys.SESSION_ID, session.sessionUUID)}. " + - log"This may indicate duplicate artifact addition.") - oldBlock.release(blockManager) + hash = hash) + // If the exact same block (same CacheId) already exists, skip re-adding. + // This prevents incorrectly removing the existing block from BlockManager. + // Note: We only skip if the CacheId matches - if it's a different session's block + // (e.g., after clone), we should replace it. + val existingBlock = hashToCachedIdMap.get(hash) + if (existingBlock == null || existingBlock.id != blockId) { + val updater = blockManager.TempFileBasedBlockStoreUpdater( + blockId = blockId, + level = StorageLevel.MEMORY_AND_DISK_SER, + classTag = implicitly[ClassTag[Array[Byte]]], + tmpFile = tmpFile, + blockSize = tmpFile.length(), + tellMaster = false) + updater.save() + hashToCachedIdMap.put(blockId.hash, new RefCountedCacheId(blockId)) + if (existingBlock != null) { + // Release the old block - this is a legitimate replacement (different CacheId, + // e.g., after session clone). The old block will be removed when its ref count + // reaches zero. + existingBlock.release(blockManager) + } + } else { + logWarning(s"Cache artifact with hash $hash already exists in this session, skipping.") } }(finallyBlock = { tmpFile.delete() }) } else if (normalizedRemoteRelativePath.startsWith(s"classes${File.separator}")) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala index ead2d52edff34..41ef4ac00c690 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala @@ -51,6 +51,18 @@ class ArtifactManagerSuite extends SharedSparkSession { super.afterEach() } + private def isBlockRegistered(id: CacheId): Boolean = { + sparkContext.env.blockManager.getStatus(id).isDefined + } + + private def addCachedArtifact(session: SparkSession, name: String, data: String): CacheId = { + val bytes = new Artifact.InMemory(data.getBytes(StandardCharsets.UTF_8)) + session.artifactManager.addLocalArtifacts(Artifact.newCacheArtifact(name, bytes) :: Nil) + val id = CacheId(session.sessionUUID, name) + assert(isBlockRegistered(id)) + id + } + test("Class artifacts are added to the correct directory.") { assume(artifactPath.resolve("smallClassFile.class").toFile.exists) @@ -534,18 +546,6 @@ class ArtifactManagerSuite extends SharedSparkSession { } test("Share blocks between ArtifactManagers") { - def isBlockRegistered(id: CacheId): Boolean = { - sparkContext.env.blockManager.getStatus(id).isDefined - } - - def addCachedArtifact(session: SparkSession, name: String, data: String): CacheId = { - val bytes = new Artifact.InMemory(data.getBytes(StandardCharsets.UTF_8)) - session.artifactManager.addLocalArtifacts(Artifact.newCacheArtifact(name, bytes) :: Nil) - val id = CacheId(session.sessionUUID, name) - assert(isBlockRegistered(id)) - id - } - // Create fresh session so there is no interference with other tests. val session1 = spark.newSession() val b1 = addCachedArtifact(session1, "b1", "b_one") @@ -582,6 +582,42 @@ class ArtifactManagerSuite extends SharedSparkSession { assert(!isBlockRegistered(b2a)) } + test("cache artifact deduplication and replacement across sessions") { + val session1 = spark.newSession() + val b1 = addCachedArtifact(session1, "b1", "data_one") + + // Add the same block again to verify that it is still registered + addCachedArtifact(session1, "b1", "data_one") + assert(isBlockRegistered(b1)) + + val session2 = session1.cloneSession() + assert(session2.artifactManager.getCachedBlockId("b1").get == b1) + + /* + * Replace the block with different data in the cloned session + * If we try to add the same hash in the cloned session, that is allowed + * and the old reference from the cloned session is removed. + */ + val b1a = addCachedArtifact(session2, "b1", "data_one_replaced") + assert(session2.artifactManager.getCachedBlockId("b1").get == b1a) + + // Verify that the original block is still registered + assert(isBlockRegistered(b1)) + assert(session1.artifactManager.getCachedBlockId("b1").get == b1) + + // Add the same block again to verify that it is still registered + addCachedArtifact(session2, "b1", "data_one_replaced") + assert(isBlockRegistered(b1a)) + + // Clean up the sessions + session1.artifactManager.cleanUpResourcesForTesting() + assert(!isBlockRegistered(b1)) + assert(isBlockRegistered(b1a)) + + session2.artifactManager.cleanUpResourcesForTesting() + assert(!isBlockRegistered(b1a)) + } + test("Codegen cache should be invalid when artifacts are added - class artifact") { withTempDir { dir => runCodegenTest("class artifact") { From 5dfa4934acfb67d1b8b25024e74f060001463b81 Mon Sep 17 00:00:00 2001 From: Alex Khakhlyuk Date: Tue, 20 Jan 2026 18:22:40 +0100 Subject: [PATCH 360/400] [SPARK-55071][CONNECT][PYTHON] Make spark.addArtifact work with Windows paths MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Currently, `spark.addArtifact` in pyspark connect does not support absolute Windows paths. E.g. this code ``` spark.addArtifact("C:\\path\\to\\file.py", pyfile=True) ``` will result in the following error ``` PySparkRuntimeError: [UNSUPPORTED_OPERATION] c scheme is not supported. ``` This error is caused by the `urlparse` function in [artifact.py.](https://github.com/apache/spark/blob/ac13473fff64919e8e7756e3a42ce3a68627dd73/python/pyspark/sql/connect/client/artifact.py#L188). It incorrectly interprets local Windows path, e.g. `C:\\path\\to\\file`, as a URI with 'C' scheme and throws an error because this URI scheme is not known and not supported. The fix is to detect absolute windows paths before calling `urlparse`, to convert them to a `file://` URI, for example `"C:\\Users\\alex.khakhlyuk\\test.py"` -> `"file:///C:/Users/alex.khakhlyuk/test.py"`, and to proceed parsing the `URI` with urlparse as for regular paths and URIs. ### Why are the changes needed? `spark.addArtifact` currently doesn't support absolute Windows paths, this should be fixed. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I ran a test locally on a windows machine. ``` bin/pyspark --remote "local[*]" spark.addArtifact("C:\\Users\\alex.khakhlyuk\\test.py", pyfile=True) ``` before: ``` PySparkRuntimeError: [UNSUPPORTED_OPERATION] c scheme is not supported. ``` after: - artifact upload succeeds It is quite tricky to write a test for windows path handling that will run on a linux CI machine. You can't create a real absolute Windows path on a Linux machine. If you run tests with a non-existent absolute Windows path, `Path(windows_path).resolve().as_uri()` will intrerpret it as a relative path and will resolve it to `{cwd}/C:\\...` which also doesn't exist. I could create some test with a bunch of mocking but I doubt how useful that would be. Existing tests make sure that we don't break existing behaviour. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53834 from khakhlyuk/fix-addartifact-on-windows. Authored-by: Alex Khakhlyuk Signed-off-by: Herman van Hövell (cherry picked from commit 888fb67699ca936ef302b1924e8e6fa63dd68b34) Signed-off-by: Herman van Hövell --- python/pyspark/sql/connect/client/artifact.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/connect/client/artifact.py b/python/pyspark/sql/connect/client/artifact.py index 72a6ffa8bf68b..a37642186fdaf 100644 --- a/python/pyspark/sql/connect/client/artifact.py +++ b/python/pyspark/sql/connect/client/artifact.py @@ -29,7 +29,7 @@ from itertools import chain from typing import List, Iterable, BinaryIO, Iterator, Optional, Tuple import abc -from pathlib import Path +from pathlib import Path, PureWindowsPath from urllib.parse import urlparse from urllib.request import url2pathname from functools import cached_property @@ -184,7 +184,17 @@ def __init__( def _parse_artifacts( self, path_or_uri: str, pyfile: bool, archive: bool, file: bool ) -> List[Artifact]: - # Currently only local files with .jar extension is supported. + # Handle Windows absolute paths (e.g., C:\path\to\file) which urlparse + # incorrectly interprets as having URI scheme 'C' instead of being a local path. + # First check if path_or_uri is a Windows path, if so, convert it to file:// URI. + try: + win_path = PureWindowsPath(path_or_uri) + if win_path.is_absolute() and win_path.drive: + # Convert Windows path to file:// URI so urlparse handles it correctly + path_or_uri = Path(path_or_uri).resolve().as_uri() + except Exception: + pass + parsed = urlparse(path_or_uri) # Check if it is a file from the scheme if parsed.scheme == "": From 05166b2aec2d46556a4baac73a71c98b0665431a Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 21 Jan 2026 07:16:21 +0900 Subject: [PATCH 361/400] [SPARK-55098][PYTHON] Vectorized UDFs with output batch size control fail with memory leak ### What changes were proposed in this pull request? Fix a memory leak that when the output stream is stopped before EOS Currently, memory is freed by `processor.close()` at EOS, but when python node is followed by a `Limit` node, the `processor.close()` is not called at the end of task and cause memory leak. ### Why are the changes needed? ``` import pyarrow as pa spark.conf.set("spark.sql.execution.arrow.maxBytesPerOutputBatch", "3") def get_size(iterator): for batch in iterator: if batch.num_rows > 0: yield pa.RecordBatch.from_arrays([pa.array([batch.num_rows])], names=['size']) spark.range(10).mapInArrow(get_size, "size long").limit(1).collect() ``` fails with ``` 26/01/20 17:45:48 ERROR BaseAllocator: Memory was leaked by query. Memory leaked: (12) Allocator(stdin reader for /Users/ruifeng.zheng/.dev/miniconda3/envs/spark-dev-313/bin/python) 0/12/12/9223372036854775807 (res/actual/peak/limit) 26/01/20 17:45:48 ERROR TaskContextImpl: Error in TaskCompletionListener java.lang.IllegalStateException: Memory was leaked by query. Memory leaked: (12) Allocator(stdin reader for /Users/ruifeng.zheng/.dev/miniconda3/envs/spark-dev-313/bin/python) 0/12/12/9223372036854775807 (res/actual/peak/limit) at org.apache.arrow.memory.BaseAllocator.close(BaseAllocator.java:504) at org.apache.spark.sql.execution.python.PythonArrowOutput$$anon$1.$anonfun$new$1(PythonArrowOutput.scala:81) at org.apache.spark.sql.execution.python.PythonArrowOutput$$anon$1.$anonfun$new$1$adapted(PythonArrowOutput.scala:77) at org.apache.spark.TaskContext$$anon$1.onTaskCompletion(TaskContext.scala:146) at org.apache.spark.TaskContextImpl.$anonfun$invokeTaskCompletionListeners$1(TaskContextImpl.scala:157) at org.apache.spark.TaskContextImpl.$anonfun$invokeTaskCompletionListeners$1$adapted(TaskContextImpl.scala:157) at org.apache.spark.TaskContextImpl.invokeListeners(TaskContextImpl.scala:212) at org.apache.spark.TaskContextImpl.invokeTaskCompletionListeners(TaskContextImpl.scala:157) at org.apache.spark.TaskContextImpl.markTaskCompleted(TaskContextImpl.scala:146) at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:199) at org.apache.spark.scheduler.Task.run(Task.scala:147) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:716) at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:86) at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:83) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:96) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:719) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642) at java.base/java.lang.Thread.run(Thread.java:1583) 26/01/20 17:45:48 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) org.apache.spark.util.TaskCompletionListenerException: Memory was leaked by query. Memory leaked: (12) Allocator(stdin reader for /Users/ruifeng.zheng/.dev/miniconda3/envs/spark-dev-313/bin/python) 0/12/12/9223372036854775807 (res/actual/peak/limit) ``` If I remove the `limit(1)`, then ``` spark.range(10).mapInArrow(get_size, "size long").collect() ``` works as expected. ### Does this PR introduce _any_ user-facing change? yes, bug-fix after this fix ``` In [2]: spark.range(10).mapInArrow(get_size, "size long").limit(1).collect() Out[2]: [Row(size=1)] ``` ### How was this patch tested? added tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #53867 from zhengruifeng/test_output_memory. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/arrow/test_arrow_map.py | 10 ++++++++++ .../spark/sql/execution/python/PythonArrowOutput.scala | 4 ++++ 2 files changed, 14 insertions(+) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_map.py b/python/pyspark/sql/tests/arrow/test_arrow_map.py index 31851e8a8adb9..2921023db4bef 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_map.py @@ -55,6 +55,16 @@ def func(iterator): expected = df.collect() self.assertEqual(actual, expected) + def test_map_in_arrow_with_limit(self): + def get_size(iterator): + for batch in iterator: + assert isinstance(batch, pa.RecordBatch) + if batch.num_rows > 0: + yield pa.RecordBatch.from_arrays([pa.array([batch.num_rows])], names=["size"]) + + df = self.spark.range(100) + df.mapInArrow(get_size, "size long").limit(1).collect() + def test_multiple_columns(self): data = [(1, "foo"), (2, None), (3, "bar"), (4, "bar")] df = self.spark.createDataFrame(data, "a int, b string") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala index 018619a5207df..df458fa37d7f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala @@ -75,6 +75,9 @@ private[python] trait PythonArrowOutput[OUT <: AnyRef] { self: BasePythonRunner[ private var processor: ArrowOutputProcessor = _ context.addTaskCompletionListener[Unit] { _ => + if (processor != null) { + processor.close() + } if (reader != null) { reader.close(false) } @@ -241,6 +244,7 @@ abstract class BaseSliceArrowOutputProcessor( prevVectors.foreach(_.close()) prevRoot.close() } + super.close() } } From d8a11bc5c07a45e39e0b76a43708322aa9ed49c9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 21 Jan 2026 09:24:31 +0900 Subject: [PATCH 362/400] [SPARK-55103][CORE][TESTS] Fix a flaky test in `RpcIntegrationSuite` ### What changes were proposed in this pull request? This PR aims to fix a flaky test in `RpcIntegrationSuite`. Recently, `sendRpcWithStreamFailures` occasionally fails. https://github.com/apache/spark/actions/runs/21073107447/job/60608256322 ``` [error] Test org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures failed: org.opentest4j.AssertionFailedError: Got a non-empty set [Failed to send RPC RPC 4622017872898699852 to localhost/127.0.0.1:41507: io.netty.channel.unix.Errors$NativeIoException: send(..) failed with error(-32): Broken pipe] ==> expected: but was: , took 0.018s [error] at org.apache.spark.network.RpcIntegrationSuite.assertErrorAndClosed(RpcIntegrationSuite.java:377) [error] at org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures(RpcIntegrationSuite.java:332) [info] Test org.apache.spark.network.RpcIntegrationSuite#sendRpcWithStreamOneAtATime() started [info] Test org.apache.spark.network.RpcIntegrationSuite#sendSuccessAndFailure() started [info] Test run finished: 1 failed, 0 ignored, 10 total, 0.195s ``` The root cause seems that one of the expected error message is [java.io.IOException Broken pipe](https://github.com/apache/spark/blob/edc7c96741c4382f79956f7ff5b9077e45f92014/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java#L370) but the actual error message is ` io.netty.channel.unix.Errors$NativeIoException: send(..) failed with error(-32): Broken pipe`. Actually, the old Netty threw `IOException` directly but newer Netty wraps `IOException` in `NativeIOException`. https://github.com/netty/netty/blob/5272df455afcd1cfae3e1156d6ac958a69fca32a/transport-native-unix-common/src/main/java/io/netty/channel/unix/Errors.java#L87 ### Why are the changes needed? For test stability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Self code review. Let's see this issue no longer occurs on CI. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53870 from sarutak/fix-rpc-test-issue. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun (cherry picked from commit a28133134c4dfba0f720be30e000cc66eb42051c) Signed-off-by: Dongjoon Hyun --- .../test/java/org/apache/spark/network/RpcIntegrationSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index e229e32e91717..e76e843b053b4 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -367,7 +367,7 @@ private void assertErrorAndClosed(RpcResult result, String expectedError) { "Connection reset", "java.nio.channels.ClosedChannelException", "io.netty.channel.StacklessClosedChannelException", - "java.io.IOException: Broken pipe" + "Broken pipe" ); Set containsAndClosed = new HashSet<>(Set.of(expectedError)); containsAndClosed.addAll(possibleClosedErrors); From 1db5ebe715629b4b622a61512d565f12aca3c8ab Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 22 Jan 2026 15:21:22 +0800 Subject: [PATCH 363/400] [SPARK-55128][INFRA] Restore SQL tests by pin 'pandas==2.3.3' ### What changes were proposed in this pull request? Restore Restore SQL tests by pin 'pandas<3' ### Why are the changes needed? pandas 3 is just released, and fail sql tests https://github.com/apache/spark/actions/runs/21232213791/job/61092886134 currently pandas 3 doesn't affect python tests too much: 1, in `dev/requirements.txt`, the latest `mlflow==3.8.1` requires: `pandas<3` 2, `pandas==2.3.3` is pinned in most places ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53910 from zhengruifeng/restore_sql. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit dafb2cd2447d35e7a41f9567651c012303fd4241) Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index f0d2acbacff7f..5735dc6ad2088 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -361,7 +361,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') || contains(matrix.modules, 'yarn') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'zstandard==0.25.0' + python3.11 -m pip install 'numpy>=1.22' pyarrow 'pandas==2.3.3' pyyaml scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'zstandard==0.25.0' python3.11 -m pip list # Run the tests. - name: Run tests From 149e15589b65139f79453ba4325ab8a44bfae28e Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 23 Jan 2026 16:07:36 +0800 Subject: [PATCH 364/400] [SPARK-55128][INFRA][FOLLOWUP] Restore SQL tests by pin 'pandas==2.3.3' for maven daily test ### What changes were proposed in this pull request? Similar to https://github.com/apache/spark/pull/53910, this pr pins the pandas version to 2.3.3. ### Why are the changes needed? To restore SQL tests for maven daily test. - https://github.com/apache/spark/actions/runs/21249870076/job/61148348328 ``` - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF *** FAILED *** udf/postgreSQL/udf-case.sql - Scalar Pandas UDF Python: 3.11 Pandas: 3.0.0 PyArrow: 23.0.0 Expected Some("struct"), but got Some("struct<>") Schema did not match for query #30 SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b WHERE udf(COALESCE(f,b.i) = 2): -- !query SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b WHERE udf(COALESCE(f,b.i) = 2) -- !query schema struct<> -- !query output org.apache.spark.SparkRuntimeException { "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "22018", "messageParameters" : { "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'nan'", "sourceType" : "\"STRING\"", "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 62, "stopIndex" : 85, "fragment" : "udf(COALESCE(f,b.i) = 2)" } ] } (SQLQueryTestSuite.scala:681) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? monitor maven daily test after pr merged ### Was this patch authored or co-authored using generative AI tooling? No Closes #53933 from LuciferYang/SPARK-55128-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: Ruifeng Zheng (cherry picked from commit 3f1c9a37a53cdc45eef173b343c0526138927c3d) Signed-off-by: yangjie01 --- .github/workflows/maven_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 74bd818a522dc..7828c4bdc6ed9 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -176,7 +176,7 @@ jobs: - name: Install Python packages (Python 3.11) if: contains(matrix.modules, 'resource-managers#yarn') || (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.22' pyarrow pandas pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'zstandard==0.25.0' + python3.11 -m pip install 'numpy>=1.22' pyarrow 'pandas==2.3.3' pyyaml scipy unittest-xml-reporting 'grpcio==1.76.0' 'grpcio-status==1.76.0' 'protobuf==6.33.0' 'zstandard==0.25.0' python3.11 -m pip list # Run the tests using script command. # BSD's script command doesn't support -c option, and the usage is different from Linux's one. From 5ad91202b33681415912ddd2f99f588ee4766b67 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 23 Jan 2026 17:38:40 +0800 Subject: [PATCH 365/400] [SPARK-55115][INFRA][4.1] Use composable Dockerfile for release builds ### What changes were proposed in this pull request? This PR refactors the release Docker image build process to use a composable Dockerfile approach: 1. **`Dockerfile.base`**: A shared base image containing common tools (Ubuntu 22.04, R packages, Ruby/bundler, TeX, Node.js) 2. **`Dockerfile`**: Branch-specific image that extends the base with Java/Python versions and packages for this branch 3. **`do-release-docker.sh`**: Updated to build the base image first, then the branch-specific image ### Why are the changes needed? Currently, each branch maintains its own full Dockerfile which leads to: - Duplicated common configuration across branches - Difficulty keeping base tools (R packages, Ruby, etc.) in sync - Expired GPG keys or outdated base images affecting all branches With the composable approach: - Common tools are defined once in `Dockerfile.base` - Each branch only specifies its unique Java/Python requirements - Updates to base tools can be applied consistently ### Version changes This is a pure refactor. No package versions were changed. ### Does this PR introduce _any_ user-facing change? No. This only affects the release infrastructure. ### How was this patch tested? Docker image built and verified successfully on remote machine. ### Was this patch authored or co-authored using generative AI tooling? Yes Closes #53906 from cloud-fan/release-infra-4.1. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- dev/create-release/do-release-docker.sh | 5 + dev/create-release/spark-rm/Dockerfile | 123 ++++++-------------- dev/create-release/spark-rm/Dockerfile.base | 110 +++++++++++++++++ 3 files changed, 152 insertions(+), 86 deletions(-) create mode 100644 dev/create-release/spark-rm/Dockerfile.base diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index 3a395e3c266bd..e231d7a48eec0 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -120,6 +120,11 @@ GPG_KEY_FILE="$WORKDIR/gpg.key" fcreate_secure "$GPG_KEY_FILE" $GPG --export-secret-key --armor --pinentry-mode loopback --passphrase "$GPG_PASSPHRASE" "$GPG_KEY" > "$GPG_KEY_FILE" +# Build base image first (contains common tools shared across all branches) +run_silent "Building spark-rm-base image..." "docker-build-base.log" \ + docker build -t "spark-rm-base:latest" -f "$SELF/spark-rm/Dockerfile.base" "$SELF/spark-rm" + +# Build branch-specific image (extends base with Java/Python versions for this branch) run_silent "Building spark-rm image with tag $IMGTAG..." "docker-build.log" \ docker build -t "spark-rm:$IMGTAG" --build-arg UID=$UID "$SELF/spark-rm" diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index e7a11d48c33f1..32a2879e229d5 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -15,112 +15,63 @@ # limitations under the License. # -# Image for building Spark releases. Based on Ubuntu 22.04. -FROM ubuntu:jammy-20250819 -LABEL org.opencontainers.image.authors="Apache Spark project " -LABEL org.opencontainers.image.licenses="Apache-2.0" -LABEL org.opencontainers.image.ref.name="Apache Spark Release Manager Image" -# Overwrite this label to avoid exposing the underlying Ubuntu OS version label -LABEL org.opencontainers.image.version="" +# Spark 4.1 release image +# Extends the base image with: +# - Java 17 +# - Python 3.10 with required packages -ENV FULL_REFRESH_DATE=20250819 - -ENV DEBIAN_FRONTEND=noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN=true +FROM spark-rm-base:latest +# Install Java 17 for Spark 4.x RUN apt-get update && apt-get install -y \ - build-essential \ - ca-certificates \ - curl \ - gfortran \ - git \ - subversion \ - gnupg \ - libcurl4-openssl-dev \ - libfontconfig1-dev \ - libfreetype6-dev \ - libfribidi-dev \ - libgit2-dev \ - libharfbuzz-dev \ - libjpeg-dev \ - liblapack-dev \ - libopenblas-dev \ - libpng-dev \ - libpython3-dev \ - libssl-dev \ - libtiff5-dev \ - libwebp-dev \ - libxml2-dev \ - msmtp \ - nodejs \ - npm \ openjdk-17-jdk-headless \ - pandoc \ - pkg-config \ + && rm -rf /var/lib/apt/lists/* + +# Install Python 3.10 +RUN apt-get update && apt-get install -y \ python3.10 \ + python3.10-dev \ python3-psutil \ - texlive-latex-base \ - texlive \ - texlive-fonts-extra \ - texinfo \ - texlive-latex-extra \ - qpdf \ - jq \ - r-base \ - ruby \ - ruby-dev \ - software-properties-common \ - wget \ - zlib1g-dev \ + libpython3-dev \ && rm -rf /var/lib/apt/lists/* +# Install pip for Python 3.10 +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 -RUN echo 'deb https://cloud.r-project.org/bin/linux/ubuntu jammy-cran40/' >> /etc/apt/sources.list -RUN gpg --keyserver hkps://keyserver.ubuntu.com --recv-key E298A3A825C0D65DFD57CBB651716619E084DAB9 -RUN gpg -a --export E084DAB9 | apt-key add - -RUN add-apt-repository 'deb https://cloud.r-project.org/bin/linux/ubuntu jammy-cran40/' - -# See more in SPARK-39959, roxygen2 < 7.2.1 -RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ - 'rmarkdown', 'testthat', 'devtools', 'e1071', 'survival', 'arrow', \ - 'ggplot2', 'mvtnorm', 'statmod', 'xml2'), repos='https://cloud.r-project.org/')" && \ - Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='https://cloud.r-project.org')" && \ - Rscript -e "devtools::install_version('lintr', version='2.0.1', repos='https://cloud.r-project.org')" && \ - Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" && \ - Rscript -e "devtools::install_version('pkgdown', version='2.0.1', repos='https://cloud.r-project.org')" - -# See more in SPARK-39735 -ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" +# Basic Python packages for Spark 4.1 +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 \ + mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" -ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 \ + googleapis-common-protos==1.71.0 graphviz==0.20.3" # Install Python 3.10 packages -RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 -RUN python3.10 -m pip install --ignore-installed 'blinker>=1.6.2' # mlflow needs this -RUN python3.10 -m pip install --ignore-installed 'six==1.16.0' # Avoid `python3-six` installation -RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ +RUN python3.10 -m pip install --ignore-installed 'blinker>=1.6.2' && \ + python3.10 -m pip install --ignore-installed 'six==1.16.0' && \ + python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ python3.10 -m pip install 'torch<2.6.0' torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.10 -m pip install deepspeed torcheval && \ python3.10 -m pip cache purge +# Sphinx and documentation packages # Should unpin 'sphinxcontrib-*' after upgrading sphinx>5 -# See 'ipython_genutils' in SPARK-38517 -# See 'docutils<0.18.0' in SPARK-39421 -RUN python3.10 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ -ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ -'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ -'pandas-stubs==1.2.0.53' 'grpcio==1.76.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ -'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' -RUN python3.10 -m pip list - -RUN gem install --no-document "bundler:2.4.22" -RUN ln -s "$(which python3.10)" "/usr/local/bin/python" -RUN ln -s "$(which python3.10)" "/usr/local/bin/python3" +# See 'ipython_genutils' in SPARK-38517, 'docutils<0.18.0' in SPARK-39421 +RUN python3.10 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' \ + sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ + ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow pandas \ + 'plotly>=4.8' 'docutils<0.18.0' 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' \ + 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' 'pandas-stubs==1.2.0.53' \ + 'grpcio==1.76.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' \ + 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' \ + 'sphinxcontrib-serializinghtml==1.1.5' -WORKDIR /opt/spark-rm/output +# Set Python 3.10 as the default +RUN ln -sf "$(which python3.10)" "/usr/local/bin/python" && \ + ln -sf "$(which python3.10)" "/usr/local/bin/python3" +# Create user for release manager ARG UID RUN useradd -m -s /bin/bash -p spark-rm -u $UID spark-rm USER spark-rm:spark-rm diff --git a/dev/create-release/spark-rm/Dockerfile.base b/dev/create-release/spark-rm/Dockerfile.base new file mode 100644 index 0000000000000..56e85256d52da --- /dev/null +++ b/dev/create-release/spark-rm/Dockerfile.base @@ -0,0 +1,110 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Base image for building Spark releases. Based on Ubuntu 22.04. +# This image contains common tools shared across all Spark versions: +# - Build tools (gcc, make, etc.) +# - R with pinned package versions +# - Ruby with bundler +# - TeX for documentation +# - Node.js for documentation +# +# Branch-specific Dockerfiles should use "FROM spark-rm-base:latest" and add: +# - Java version (8 or 17) +# - Python version and pip packages + +FROM ubuntu:jammy-20250819 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Release Manager Base Image" +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE=20250819 + +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true + +# Install common system packages and build tools +# Note: Java and Python are installed in branch-specific Dockerfiles +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + subversion \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libssl-dev \ + libtiff5-dev \ + libwebp-dev \ + libxml2-dev \ + msmtp \ + nodejs \ + npm \ + pandoc \ + pkg-config \ + texlive-latex-base \ + texlive \ + texlive-fonts-extra \ + texinfo \ + texlive-latex-extra \ + qpdf \ + jq \ + r-base \ + ruby \ + ruby-dev \ + software-properties-common \ + wget \ + zlib1g-dev \ + && rm -rf /var/lib/apt/lists/* + +# Set up R CRAN repository for latest R packages +RUN echo 'deb https://cloud.r-project.org/bin/linux/ubuntu jammy-cran40/' >> /etc/apt/sources.list && \ + gpg --keyserver hkps://keyserver.ubuntu.com --recv-key E298A3A825C0D65DFD57CBB651716619E084DAB9 && \ + gpg -a --export E084DAB9 | apt-key add - && \ + add-apt-repository 'deb https://cloud.r-project.org/bin/linux/ubuntu jammy-cran40/' + +# Install R packages (same versions across all branches) +# See more in SPARK-39959, roxygen2 < 7.2.1 +RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ + 'rmarkdown', 'testthat', 'e1071', 'survival', 'arrow', \ + 'ggplot2', 'mvtnorm', 'statmod', 'xml2'), repos='https://cloud.r-project.org/')" && \ + Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('lintr', version='2.0.1', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('pkgdown', version='2.0.1', repos='https://cloud.r-project.org')" + +# See more in SPARK-39735 +ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" + +# Install Ruby bundler (same version across all branches) +RUN gem install --no-document "bundler:2.4.22" + +# Create workspace directory +WORKDIR /opt/spark-rm/output + +# Note: Java, Python, and user creation are done in branch-specific Dockerfiles From 409e1fd68ff1e3c70b4be07a5127b2c382515788 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 24 Jan 2026 22:58:08 +0900 Subject: [PATCH 366/400] [SPARK-55134] Fix `BasicExecutorFeatureStep` to throw `IllegalArgumentException` for executor cpu misconfigs ### What changes were proposed in this pull request? This PR aims to fix `BasicExecutorFeatureStep` to throw `IllegalArgumentException` for executor cpu misconfigs in order the Spark jobs ASAP. ### Why are the changes needed? From Apache Spark 4.1.0, Spark driver pod throws `SparkException` for the executor cpu misconfiguration before requesting to the K8s control plain. This improvement reduces the burden of K8s control plane. - #51678 ``` 26/01/24 06:55:31 INFO ExecutorPodsAllocator: Going to request 5 executors from Kubernetes for ResourceProfile Id: 0, target: 5, known: 0, sharedSlotFromPendingPods: 2147483647. 26/01/24 06:55:31 INFO ExecutorPodsAllocator: Found 0 reusable PVCs from 0 PVCs 26/01/24 06:55:31 WARN ExecutorPodsSnapshotsStoreImpl: Exception when notifying snapshot subscriber. org.apache.spark.SparkException: The executor cpu request (4) should be less than or equal to cpu limit (1) at org.apache.spark.deploy.k8s.features.BasicExecutorFeatureStep.$anonfun$configurePod$11(BasicExecutorFeatureStep.scala:236) ``` However, the Spark driver keeps re-trying to create executor pods in any way if the users didn't have an additional `spark.driver.timeout` configuration. - #45313 So, we had better exit the Spark job in this case ASAP. We can do that simply switches `SparkException` to `IllegalArgumentException` like the other steps. - https://github.com/apache/spark/pull/30084 ### Does this PR introduce _any_ user-facing change? Technically no because previously those misconfigured Spark job didn't get any resources. ### How was this patch tested? Pass the CIs with the updated test case. Also, I checked manually via `spark-submit`: ``` $ bin/spark-submit --master k8s://$K8S_MASTER \ --deploy-mode cluster \ -c spark.executor.instances=5 \ -c spark.kubernetes.executor.request.cores=4 \ -c spark.kubernetes.executor.limit.cores=1 \ -c spark.kubernetes.container.image=apache/spark:SPARK-55134 \ -c spark.kubernetes.authenticate.driver.serviceAccountName=spark \ -c spark.kubernetes.executor.useDriverPodIP=true \ --class org.apache.spark.examples.SparkPi \ local:///opt/spark/examples/jars/spark-examples.jar 200000 ... 26/01/24 16:33:57 INFO LoggingPodStatusWatcherImpl: State changed, new state: pod name: org-apache-spark-examples-sparkpi-0482f19beeec7491-driver namespace: default labels: spark-app-name -> org-apache-spark-examples-sparkpi, spark-app-selector -> spark-ee23f03db88b43fb906b0dbc1b04ad63, spark-role -> driver, spark-version -> 4.2.0-SNAPSHOT pod uid: c6d41845-5893-4135-a065-278d94500315 creation time: 2026-01-24T07:33:52Z service account name: spark volumes: spark-local-dir-1, spark-conf-volume-driver, kube-api-access-8rbc8 node name: lima-rancher-desktop start time: 2026-01-24T07:33:52Z phase: Failed container status: container name: spark-kubernetes-driver container image: apache/spark:SPARK-55134 container state: terminated container started at: 2026-01-24T07:33:53Z container finished at: 2026-01-24T07:33:55Z exit code: 1 termination reason: Error 26/01/24 16:33:57 INFO LoggingPodStatusWatcherImpl: Application status for spark-ee23f03db88b43fb906b0dbc1b04ad63 (phase: Failed) 26/01/24 16:33:57 INFO LoggingPodStatusWatcherImpl: Container final statuses: container name: spark-kubernetes-driver container image: apache/spark:SPARK-55134 container state: terminated container started at: 2026-01-24T07:33:53Z container finished at: 2026-01-24T07:33:55Z exit code: 1 termination reason: Error 26/01/24 16:33:57 INFO LoggingPodStatusWatcherImpl: Application org.apache.spark.examples.SparkPi with application ID spark-ee23f03db88b43fb906b0dbc1b04ad63 and submission ID default:org-apache-spark-examples-sparkpi-0482f19beeec7491-driver finished ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53948 from dongjoon-hyun/SPARK-55134. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit ab3ec9e34441dc11372e764d6384b309af36abee) Signed-off-by: Dongjoon Hyun --- .../spark/deploy/k8s/features/BasicExecutorFeatureStep.scala | 4 ++-- .../deploy/k8s/features/BasicExecutorFeatureStepSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 13d1f1bc98a0e..0cfa842ef3963 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -232,8 +232,8 @@ private[spark] class BasicExecutorFeatureStep( executorLimitCores.map { limitCores => val executorCpuLimitQuantity = new Quantity(limitCores) if (executorCpuLimitQuantity.compareTo(executorCpuQuantity) < 0) { - throw new SparkException(s"The executor cpu request ($executorCpuQuantity) should be " + - s"less than or equal to cpu limit ($executorCpuLimitQuantity)") + throw new IllegalArgumentException(s"The executor cpu request ($executorCpuQuantity) " + + s"should be less than or equal to cpu limit ($executorCpuLimitQuantity)") } new ContainerBuilder(executorContainerWithConfVolume) .editResources() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index ced1326e7938c..d264484f4d039 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -123,7 +123,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { test("SPARK-52933: Verify if the executor cpu request exceeds limit") { baseConf.set(KUBERNETES_EXECUTOR_REQUEST_CORES, "2") baseConf.set(KUBERNETES_EXECUTOR_LIMIT_CORES, "1") - val error = intercept[SparkException] { + val error = intercept[IllegalArgumentException] { initDefaultProfile(baseConf) val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf), defaultProfile) From f314f1682c154dcaf91a313139b6357293fa157a Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Tue, 27 Jan 2026 19:51:49 +0800 Subject: [PATCH 367/400] [SPARK-48455][DOCS][MINOR] Remove unused DECLARE statement from IF statement example in Docs ### What changes were proposed in this pull request? Remove dead code from example in If Statement documentation. ### Why are the changes needed? It is needed to keep documentation in good shape. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No tests needed. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53995 from miland-db/milan-dankovic_data/improve-docs. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan (cherry picked from commit 61d4581b7b2ada0efbf93f682c39c73b9252df79) Signed-off-by: Wenchen Fan --- docs/control-flow/if-stmt.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/control-flow/if-stmt.md b/docs/control-flow/if-stmt.md index 2e93dc7f6d505..a687fc5c1c643 100644 --- a/docs/control-flow/if-stmt.md +++ b/docs/control-flow/if-stmt.md @@ -47,7 +47,6 @@ IF condition THEN { stmt ; } [...] ```SQL > BEGIN DECLARE choice DOUBLE DEFAULT 3.9; - DECLARE result STRING; IF choice < 2 THEN VALUES ('one fish'); ELSEIF choice < 3 THEN From cd4a6dcb6f13bc5a159e6f3d02d5fb9f60cc9d6b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 27 Jan 2026 08:27:08 -0800 Subject: [PATCH 368/400] [SPARK-55223][DOCS] Document sinks in declarative pipelines programming guide ### What changes were proposed in this pull request? Adds a section on sinks to the declarative pipelines programming guide page in the docs ### Why are the changes needed? The SDP programming guide should cover all SDP features ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #53991 from sryza/sink-docs. Authored-by: Sandy Ryza Signed-off-by: Sandy Ryza (cherry picked from commit 55061e9c2a589aeb067cf48857167a5a0136caf3) Signed-off-by: Sandy Ryza --- ...declarative-pipelines-programming-guide.md | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/docs/declarative-pipelines-programming-guide.md b/docs/declarative-pipelines-programming-guide.md index 859258a430dd6..5b3a06fe26c0e 100644 --- a/docs/declarative-pipelines-programming-guide.md +++ b/docs/declarative-pipelines-programming-guide.md @@ -449,6 +449,52 @@ AS INSERT INTO customers_us SELECT * FROM STREAM(customers_us_east); ``` +## Writing Data to External Targets with Sinks + +Sinks in SDP provide a way to write transformed data to external destinations beyond the default streaming tables and materialized views. Sinks are particularly useful for operational use cases that require low-latency data processing, reverse ETL operations, or writing to external systems. + +Sinks enable a pipeline to write to any destination that a Spark Structured Streaming query can be written to, including, but not limited to, **Apache Kafka** and **Azure Event Hubs**. + +### Creating and Using Sinks in Python + +Working with sinks involves two main steps: creating the sink definition and implementing an append flow to write data. + +#### Creating a Kafka Sink + +You can create a sink that streams data to a Kafka topic: + +```python +from pyspark import pipelines as dp +from pyspark.sql.functions import to_json, struct + +dp.create_sink( + name="kafka_sink", + format="kafka", + options={ + "kafka.bootstrap.servers": "localhost:9092", + "topic": "processed_orders" + } +) + +@dp.append_flow(target="kafka_sink") +def kafka_orders_flow() -> DataFrame: + return ( + spark.readStream.table("customer_orders") + .select( + col("order_id").cast("string").alias("key"), + to_json(struct("*")).alias("value") + ) + ) +``` + +### Sink Considerations + +When working with sinks, keep the following considerations in mind: + +- **Streaming-only**: Sinks currently support only streaming queries through `append_flow` decorators +- **Python API**: Sink functionality is available only through the Python API, not SQL +- **Append-only**: Only append operations are supported; full refresh updates reset checkpoints but do not clean previously computed results + ## Important Considerations ### Python Considerations From a2fa28dcd35554b0fc28992944b560e27fb9c572 Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Wed, 28 Jan 2026 20:41:06 +0800 Subject: [PATCH 369/400] [SPARK-55119][SQL] Fix Continue Handler: prevent INTERNAL_ERROR and incorrect conditional statements interruption ### What changes were proposed in this pull request? This PR fixes two issues with `CONTINUE` handler behavior in SQL scripting when exceptions occur in or around conditional statements: **Issue 1: INTERNAL_ERROR when CONTINUE handler interrupts conditional statement at end of loop body** - When a CONTINUE handler processes an exception in a conditional statement's condition (e.g., `IF (1/0)==1 THEN`), the conditional is interrupted - If the conditional is the last statement in a loop body (WHILE/REPEAT), the loop would call `next()` on an exhausted iterator after the handler completes - This caused: `INTERNAL_ERROR: No more elements to iterate through in the current SQL compound statement` **Fix:** Added `hasNext` checks in WHILE and REPEAT loop iterators before calling `next()` on the body. If the body is exhausted, return `NoOpStatementExec` and transition back to the condition state instead of calling `next()`. **Issue 2: Incorrect interruption of conditional statements when exception occurs before them** - When an exception occurs **before** a conditional statement (e.g., `SIGNAL SQLSTATE '02000'; WHILE ... DO`), the conditional was incorrectly interrupted - This caused the loop statement body to be skipped even though the exception didn't occur during its evaluation - The interrupt logic couldn't distinguish between "exception during evaluation" vs "exception before reaching the statement" **Fix:** Added explicit tracking of when evaluation starts: - `ForStatementExec`: Added `hasStartedQueryEvaluation` flag, set when `cachedQueryResult()` begins evaluating the FOR loop's query - `SimpleCaseStatementExec`: Added `hasStartedCaseVariableEvaluation` flag, set when `validateCache()` begins evaluating the case variable - Updated `interruptConditionalStatements()` to check these flags - only interrupt if evaluation was actually attempted ### Why are the changes needed? Cursor support is dependent on `CONTINUE HANDLER` correct behavior. We needed to fix this to unblock development of cursors support. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added comprehensive test coverage in `SqlScriptingExecutionSuite` that tests both **Issue 1** and **Issue 2**. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53894 from miland-db/milan-dankovic_data/double-advancement-bug. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan (cherry picked from commit cecf758b4323417a894c37641d760c723b530015) Signed-off-by: Wenchen Fan --- .../sql/scripting/SqlScriptingExecution.scala | 42 ++- .../scripting/SqlScriptingExecutionNode.scala | 41 +++ .../SqlScriptingExecutionSuite.scala | 338 ++++++++++++++++++ 3 files changed, 415 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala index c8f7172e59bd7..1dee2ae80f508 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala @@ -88,7 +88,8 @@ class SqlScriptingExecution( /** * Helper method to execute interrupts to ConditionalStatements. - * This method should only interrupt when the statement that throws is a conditional statement. + * This method should only interrupt when the exception was thrown during evaluation of + * the conditional statement's condition. * @param executionPlan Execution plan. */ private def interruptConditionalStatements(executionPlan: NonLeafStatementExec): Unit = { @@ -101,11 +102,40 @@ class SqlScriptingExecution( } currExecPlan match { - case exec: ConditionalStatementExec => - // Only interrupt if the conditional statement is currently evaluating its condition. - // For loop statements, this means we should skip the loop when an exception occurs - // during condition evaluation, but NOT when an exception occurs in the loop body. - if (exec.isInCondition) { + case exec: ConditionalStatementExec if exec.isInCondition => + // Only interrupt the conditional if its condition/query was being evaluated when the + // exception occurred. This distinguishes between two scenarios: + // 1. Exception during condition evaluation -> interrupt (skip the conditional) + // 2. Exception before reaching the conditional -> don't interrupt (execute normally) + // + // Different conditional statements track evaluation state differently: + // - SimpleCaseStatementExec: hasStartedCaseVariableEvaluation flag is set when + // validateCache() begins evaluating the case variable expression. + // - ForStatementExec: hasStartedQueryEvaluation flag is set when cachedQueryResult() + // begins evaluating the FOR loop's query. + // - IF/ELSEIF, WHILE, REPEAT, SEARCHED CASE: curr.isExecuted flag is set by + // evaluateBooleanCondition() before evaluating each condition. + val shouldInterrupt = + exec match { + case simpleCaseStmt: SimpleCaseStatementExec + if simpleCaseStmt.hasStartedCaseVariableEvaluation => + // Only interrupt if case variable evaluation was attempted. + true + case forStmt: ForStatementExec => + // Only interrupt if query evaluation was attempted. + forStmt.hasStartedQueryEvaluation + case _ => + // For IF, WHILE, REPEAT, SEARCHED/SIMPLE CASE: check if condition was executed. + // evaluateBooleanCondition sets isExecuted=true before evaluation, so if an + // exception occurs during evaluation, isExecuted will be true. If the exception + // happened before reaching the conditional, isExecuted will still be false. + exec.curr match { + case Some(stmt: SingleStatementExec) => stmt.isExecuted + case _ => false + } + } + + if (shouldInterrupt) { exec.interrupted = true } case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index c47df4b7a89ea..953301ab8ed54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -554,6 +554,18 @@ class WhileStatementExec( throw SparkException.internalError("Unexpected statement type in WHILE condition.") } case WhileState.Body => + // Check if body has more statements before calling next(). When an exception in a + // conditional statement's condition is handled by a CONTINUE handler, the conditional + // is interrupted. If it's the last statement in the loop body, calling next() on the + // exhausted iterator would fail. Instead, we return NoOpStatementExec and transition + // back to the condition. + if (!body.getTreeIterator.hasNext) { + state = WhileState.Condition + curr = Some(condition) + condition.reset() + return new NoOpStatementExec + } + val retStmt = body.getTreeIterator.next() // Handle LEAVE or ITERATE statement if it has been encountered. @@ -716,15 +728,23 @@ class SimpleCaseStatementExec( private var conditionBodyTupleIterator: Iterator[(SingleStatementExec, CompoundBodyExec)] = _ private var caseVariableLiteral: Literal = _ + // Flag to track if case variable evaluation has been attempted. Used by CONTINUE handler + // mechanism to determine if an exception occurred during case variable evaluation vs. before + // the CASE statement was reached. + protected[scripting] var hasStartedCaseVariableEvaluation: Boolean = false + private var isCacheValid = false private def validateCache(): Unit = { if (!isCacheValid) { + // Set flags before evaluation so CONTINUE handler can detect if exception happened here. + hasStartedCaseVariableEvaluation = true val values = caseVariableExec.buildDataFrame(session).collect() caseVariableExec.isExecuted = true caseVariableLiteral = Literal(values.head.get(0)) conditionBodyTupleIterator = createConditionBodyIterator isCacheValid = true + hasStartedCaseVariableEvaluation = false } } @@ -826,6 +846,7 @@ class SimpleCaseStatementExec( caseVariableExec.reset() conditionalBodies.foreach(b => b.reset()) elseBody.foreach(b => b.reset()) + hasStartedCaseVariableEvaluation = false } } @@ -876,6 +897,18 @@ class RepeatStatementExec( throw SparkException.internalError("Unexpected statement type in REPEAT condition.") } case RepeatState.Body => + // Check if body has more statements before calling next(). When an exception in a + // conditional statement's condition is handled by a CONTINUE handler, the conditional + // is interrupted. If it's the last statement in the loop body, calling next() on the + // exhausted iterator would fail. Instead, we return NoOpStatementExec and transition + // back to the condition. + if (!body.getTreeIterator.hasNext) { + state = RepeatState.Condition + curr = Some(condition) + condition.reset() + return new NoOpStatementExec + } + val retStmt = body.getTreeIterator.next() retStmt match { @@ -1043,17 +1076,24 @@ class ForStatementExec( } private var state = ForState.VariableAssignment + // Flag to track if FOR query evaluation has been attempted. Used by CONTINUE handler + // mechanism to determine if an exception occurred during query evaluation vs. before + // the FOR statement was reached. + protected[scripting] var hasStartedQueryEvaluation = false private var queryResult: util.Iterator[Row] = _ private var queryColumnNameToDataType: Map[String, DataType] = _ private var isResultCacheValid = false private def cachedQueryResult(): util.Iterator[Row] = { if (!isResultCacheValid) { + // Set flag before evaluation so CONTINUE handler can detect if exception happened here. + hasStartedQueryEvaluation = true val df = query.buildDataFrame(session) queryResult = df.toLocalIterator() queryColumnNameToDataType = df.schema.fields.map(f => f.name -> f.dataType).toMap query.isExecuted = true isResultCacheValid = true + hasStartedQueryEvaluation = false } queryResult } @@ -1250,6 +1290,7 @@ class ForStatementExec( curr = None bodyWithVariables = None firstIteration = true + hasStartedQueryEvaluation = false } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala index 2ad715f671edc..5ed84db96836a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala @@ -403,6 +403,310 @@ class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(sqlScript, expected = expected) } + test("continue handler - Exception in FOR loop query evaluation") { + val sqlScript = + """ + |BEGIN + | DECLARE i = 0; + | DECLARE handled = 0; + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | SET handled = handled + 1; + | + | WHILE i < 2 DO + | SET i = i + 1; + | FOR o AS SELECT 1/0 DO + | SELECT 1; + | END FOR; + | END WHILE; + | + | SELECT handled; + |END + |""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - WHILE loop with exception in IF condition") { + val sqlScript = + """ + |BEGIN + | DECLARE i INT DEFAULT 0; + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | SELECT 22; + | + | WHILE i < 2 DO + | SET i = i + 1; + | + | IF (1/0)==1 THEN + | SET s = s + 5; + | END IF; + | END WHILE; + | + | SELECT i, s; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed on first iteration + Seq(Row(22)), // handler executed on second iteration + Seq(Row(2, 0)) // final select: i=2, s=0 (never incremented) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - REPEAT loop with exception in IF condition") { + val sqlScript = + """ + |BEGIN + | DECLARE i INT DEFAULT 0; + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | SELECT 22; + | + | REPEAT + | SET i = i + 1; + | + | IF (1/0)==1 THEN + | SET s = s + 5; + | END IF; + | UNTIL i >= 2 + | END REPEAT; + | + | SELECT i, s; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed on first iteration + Seq(Row(22)), // handler executed on second iteration + Seq(Row(2, 0)) // final select: i=2, s=0 (never incremented) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - WHILE loop with zero division before IF") { + val sqlScript = + """ + |BEGIN + | DECLARE i INT DEFAULT 0; + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SELECT 22; + | END; + | + | WHILE i < 2 DO + | SET i = i + 1; + | + | SELECT 1/0; + | + | IF 1==1 THEN + | SET s = s + 5; + | END IF; + | END WHILE; + | + | SELECT i, s; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed on first iteration + Seq(Row(22)), // handler executed on second iteration + Seq(Row(2, 10)) // final select: i=2, s=10 (incremented twice) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - REPEAT loop with zero division before IF") { + val sqlScript = + """ + |BEGIN + | DECLARE i INT DEFAULT 0; + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SELECT 22; + | END; + | + | REPEAT + | SET i = i + 1; + | + | SELECT 1/0; + | + | IF 1==1 THEN + | SET s = s + 5; + | END IF; + | UNTIL i >= 2 + | END REPEAT; + | + | SELECT i, s; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed on first iteration + Seq(Row(22)), // handler executed on second iteration + Seq(Row(2, 10)) // final select: i=2, s=10 (incremented twice) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - FOR loop with exception in IF condition") { + val sqlScript = + """ + |BEGIN + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | SELECT 22; + | + | FOR i AS VALUES (1), (2) DO + | SELECT 1; + | IF (1/0)==1 THEN + | SET s = s + 5; + | END IF; + | END FOR; + | + | SELECT s; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1)), // SELECT 1 on first iteration + Seq(Row(22)), // handler executed on first iteration + Seq(Row(1)), // SELECT 1 on second iteration + Seq(Row(22)), // handler executed on second iteration + Seq(Row(0)) // final select: i=2, s=0 (never incremented) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - FOR loop with zero division before IF") { + val sqlScript = + """ + |BEGIN + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SELECT 22; + | END; + | + | FOR i AS VALUES (1), (2) DO + | SELECT 1; + | SELECT 1/0; + | + | IF 1==1 THEN + | SET s = s + 5; + | END IF; + | END FOR; + | + | SELECT 10; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1)), // SELECT 1 on first iteration + Seq(Row(22)), // handler executed on first iteration + Seq(Row(1)), // SELECT 1 on second iteration + Seq(Row(22)), // handler executed on second iteration + Seq(Row(10)) // final select: s=10 (incremented twice) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - zero division before FOR loop") { + val sqlScript = + """ + |BEGIN + | DECLARE s INT DEFAULT 0; + | + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SELECT 22; + | END; + | + | SELECT 1/0; + | + | FOR i AS VALUES (1), (2) DO + | SELECT 1; + | IF 1==1 THEN + | SET s = s + 5; + | END IF; + | END FOR; + | + | SELECT s; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed + Seq(Row(1)), // SELECT 1 on first iteration + Seq(Row(1)), // SELECT 1 on second iteration + Seq(Row(10)) // final select: s=10 (incremented twice) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + + test("continue handler - zero division before simple case") { + withTable("t") { + val commands = + """ + |BEGIN + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SELECT 22; + | END; + | + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | INSERT INTO t VALUES (2, 'b', 2.0); + | + | SELECT 1/0; + | CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 42; + | WHEN 3 THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END CASE; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed + Seq(Row(44)) + ) + verifySqlScriptResult(commands, expected) + } + } + + test("continue handler - zero division before searched case") { + val commands = + """ + |BEGIN + | DECLARE CONTINUE HANDLER FOR DIVIDE_BY_ZERO + | BEGIN + | SELECT 22; + | END; + | + | SELECT 1/0; + | + | CASE + | WHEN 1 = (SELECT 2) THEN + | SELECT 1; + | WHEN 2 = 2 THEN + | SELECT 42; + | WHEN (SELECT * FROM t) THEN + | SELECT * FROM b; + | END CASE; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(22)), // handler executed + Seq(Row(42)) + ) + verifySqlScriptResult(commands, expected) + } + test("handler - exit resolve in outer block") { val sqlScript = """ @@ -973,6 +1277,40 @@ class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(sqlScript, expected = expected) } + test("continue handler - continue when exception happens in simple case body") { + val sqlScript = + """ + |BEGIN + | DECLARE VARIABLE flag INT = -1; + | DECLARE VARIABLE x INT = 1; + | DECLARE CONTINUE HANDLER FOR SQLSTATE '22012' + | BEGIN + | SELECT 22; + | SET flag = 1; + | END; + | + | CASE x + | WHEN flag THEN + | SELECT 10; + | WHEN 1 THEN + | SELECT 11; + | SELECT 1/0; + | SELECT 33; + | ELSE + | SELECT 12; + | END CASE; + | SELECT flag; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(11)), + Seq(Row(22)), + Seq(Row(33)), + Seq(Row(1)) + ) + verifySqlScriptResult(sqlScript, expected = expected) + } + test("exit handler - exit resolve when simple case condition computation fails") { val sqlScript = """ From 72e892dece2634d9fbc7cfd0f073f092886a45ac Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 30 Jan 2026 22:24:07 +0800 Subject: [PATCH 370/400] [SPARK-55133][CONNECT] Fix race condition in IsolatedSessionState lifecycle management This PR fixes a race condition in `IsolatedSessionState` lifecycle management that could cause failures when `spark.executor.isolatedSessionCache.size` is set to a small value. Key changes: - Introduced `IsolatedSessionState.sessions` as the authoritative store for all isolated sessions, ensuring only one session exists per UUID at any time - Changed `refCount` and `evicted` from lock-free to synchronized access via a shared lock object to prevent race conditions between `acquire()`, `release()`, and `markEvicted()` - Added `acquire()` return value to indicate if the session was successfully acquired (returns false if already evicted) - Added `tryUnEvict()` method to allow reusing a deferred session that was evicted but still in use - Updated the cache loader to check the authoritative sessions map first and reuse existing sessions when possible When the isolated session cache is full and sessions are evicted, there's a race condition between: 1. A task acquiring a session from the cache 2. Another task triggering eviction of that session 3. The evicted session being cleaned up (classloader closed, files deleted) This could cause: - `RemoteClassLoaderError` when trying to load classes with a closed classloader - `NoSuchFileException` when session files are deleted while still in use The fix ensures that: - Sessions are tracked in an authoritative map from creation until cleanup completes - Evicted sessions can be reused if still in use by other tasks - A task cannot acquire a session that's being cleaned up No. Existing tests and a new test suite Yes, cursor 2.3 Closes #53914 from cloud-fan/session. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 1da0e5357ad95f174f5e8b637d1948a488e017ac) Signed-off-by: Wenchen Fan --- .../org/apache/spark/executor/Executor.scala | 201 +++++++++++++-- .../ExecutorSideSessionManagementSuite.scala | 233 ++++++++++++++++++ .../sql/connect/test/RemoteSparkSession.scala | 2 +- 3 files changed, 414 insertions(+), 22 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/executor/ExecutorSideSessionManagementSuite.scala diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index edab354b96078..c8843ac3427e4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -24,7 +24,7 @@ import java.net.{URI, URL, URLClassLoader} import java.nio.ByteBuffer import java.util.{Locale, Properties} import java.util.concurrent._ -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantLock import javax.annotation.concurrent.GuardedBy @@ -58,6 +58,91 @@ import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.ArrayImplicits._ +private[spark] object IsolatedSessionState { + // Authoritative store for all isolated sessions. Sessions are put here when created + // and removed when cleanup runs. The Guava cache just tracks which sessions are + // "active" for LRU eviction policy, but this map is the source of truth. + // This ensures there's only ONE IsolatedSessionState per UUID at any time. + val sessions = new ConcurrentHashMap[String, IsolatedSessionState]() +} + +/** + * Represents an isolated session state on the executor side, containing session-specific + * classloaders, files, jars, and archives. This class manages the lifecycle of these resources + * and prevents race conditions between concurrent task execution and cache eviction. + * + * == Architecture == + * + * Sessions are managed through two mechanisms: + * 1. A Guava LRU cache (`isolatedSessionCache`) for active session lookup with size limits + * 2. An authoritative map (`IsolatedSessionState.sessions`) tracking all sessions until cleanup + * + * The Guava cache handles LRU eviction, while the authoritative map ensures there's only one + * IsolatedSessionState instance per UUID at any time and tracks sessions that are evicted but + * still in use. + * + * == State Machine == + * + * Each session has two state variables protected by a synchronized lock: + * - `refCount`: Number of tasks currently using this session + * - `evicted`: Whether the session has been evicted from the Guava cache + * + * Valid state transitions: + * {{{ + * Common workflow (no contention): + * [Created] --> acquire() --> [Active: refCount > 0] + * | + * release() (all tasks done) + * | + * v + * [Idle: refCount = 0] + * | + * markEvicted() (cache eviction) + * | + * v + * [Cleanup] + * + * Contention case (eviction while tasks running): + * [Active: refCount > 0] --> markEvicted() --> [Deferred: evicted = true] + * | + * +---------------------------------------------+ + * | | + * v v + * release() (last task) tryUnEvict() + * | | + * v v + * [Cleanup] [Active] (back in cache) + * }}} + * + * == Cleanup == + * + * Cleanup happens when both conditions are met: `refCount == 0` AND `evicted == true`. + * This can occur either: + * - Immediately when `markEvicted()` is called and no tasks are using the session + * - Deferred when the last task calls `release()` after the session was evicted + * + * Cleanup closes the classloader, deletes session files, and removes the session + * from the authoritative map. + * + * == Concurrency Guarantees == + * + * The key insight is that as long as a session is still in use (refCount > 0), it + * remains in the authoritative map and we can get its instance. When a new task needs + * a session that was evicted from the LRU cache but is still in use: + * + * - If cleanup has NOT started (refCount > 0): we can cancel the pending cleanup + * via `tryUnEvict()`, put the instance back into the LRU cache, and safely reuse it. + * + * - If cleanup HAS started (refCount became 0): cleanup runs synchronously under the + * lock, so it must complete before any new task can proceed. Once cleanup finishes, + * the session is removed from the authoritative map, and a fresh instance is created. + * + * This design ensures there is never a race where a task uses a session that is being + * or has been cleaned up. The `acquire()` and `tryUnEvict()` methods are intentionally + * separate: `tryUnEvict()` is only called from the cache loader to guarantee the + * session is put back into the LRU cache, maintaining the invariant that a non-evicted + * session is always in the cache. + */ private[spark] class IsolatedSessionState( val sessionUUID: String, var urlClassLoader: MutableURLClassLoader, @@ -68,29 +153,62 @@ private[spark] class IsolatedSessionState( val replClassDirUri: Option[String]) extends Logging { // Reference count for the number of running tasks using this session. - private val refCount: AtomicInteger = new AtomicInteger(0) + // Access is synchronized via `lock`. + private var refCount: Int = 0 // Whether this session has been evicted from the cache. - @volatile private var evicted: Boolean = false + // Access is synchronized via `lock`. + private var evicted: Boolean = false - /** Increment the reference count, indicating a task is using this session. */ - def acquire(): Unit = refCount.incrementAndGet() + // Lock to synchronize all state changes. + private val lock = new Object + + /** + * Increment the reference count, indicating a task is using this session. + * @return true if the session was successfully acquired, false if it was already evicted + */ + def acquire(): Boolean = lock.synchronized { + if (evicted) { + false + } else { + refCount += 1 + true + } + } + + /** + * Try to un-evict this session so it can be reused. + * This is called from the cache loader to reuse a deferred session. + * The caller should call acquire() separately after the session is in cache. + * @return true if successfully un-evicted, false if already cleaned up or refCount is 0 + */ + def tryUnEvict(): Boolean = lock.synchronized { + if (evicted && refCount > 0) { + evicted = false + logInfo(log"Session ${MDC(SESSION_ID, sessionUUID)} un-evicted, " + + log"still in use by ${MDC(LogKeys.COUNT, refCount)} task(s)") + true + } else { + false + } + } /** Decrement the reference count. If evicted and no more tasks, clean up. */ - def release(): Unit = { - if (refCount.decrementAndGet() == 0 && evicted) { + def release(): Unit = lock.synchronized { + refCount -= 1 + if (refCount == 0 && evicted) { cleanup() } } - /** Mark this session as evicted. If no tasks are using it, clean up immediately. */ - def markEvicted(): Unit = { + /** Mark this session as evicted. Cleans up immediately if refCount is 0. */ + def markEvicted(): Unit = lock.synchronized { evicted = true - if (refCount.get() == 0) { + if (refCount == 0) { cleanup() } else { logInfo(log"Session ${MDC(SESSION_ID, sessionUUID)} evicted but still in use by " + - log"${MDC(LogKeys.COUNT, refCount.get())} task(s), deferring cleanup") + log"${MDC(LogKeys.COUNT, refCount)} task(s), deferring cleanup") } } @@ -108,11 +226,15 @@ private[spark] class IsolatedSessionState( logWarning(log"Failed to close urlClassLoader for session " + log"${MDC(SESSION_ID, sessionUUID)}", e) } + // Delete session files. val sessionBasedRoot = new File(SparkFiles.getRootDirectory(), sessionUUID) if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) { Utils.deleteRecursively(sessionBasedRoot) } + + // Remove from authoritative sessions map after cleanup + IsolatedSessionState.sessions.remove(sessionUUID) logInfo(log"Session cleaned up: ${MDC(SESSION_ID, sessionUUID)}") } } @@ -241,13 +363,17 @@ private[spark] class Executor( isDefaultState(jobArtifactState.uuid)) val replClassLoader = addReplClassLoaderIfNeeded( urlClassLoader, jobArtifactState.replClassDirUri, jobArtifactState.uuid) - new IsolatedSessionState( + val state = new IsolatedSessionState( jobArtifactState.uuid, urlClassLoader, replClassLoader, currentFiles, currentJars, currentArchives, jobArtifactState.replClassDirUri ) + // Store in the authoritative sessions map immediately. + // This ensures there's only one session per UUID at any time. + IsolatedSessionState.sessions.put(jobArtifactState.uuid, state) + state } private def isStubbingEnabledForState(name: String) = { @@ -258,7 +384,7 @@ private[spark] class Executor( private def isDefaultState(name: String) = name == "default" // Classloader isolation - // The default isolation group + // The default isolation group. Not in the cache, never evicted. val defaultSessionState: IsolatedSessionState = newSessionState(JobArtifactState("default", None)) val isolatedSessionCache: Cache[String, IsolatedSessionState] = CacheBuilder.newBuilder() @@ -270,8 +396,8 @@ private[spark] class Executor( val state = notification.getValue // Cache is always used for isolated sessions. assert(!isDefaultState(state.sessionUUID)) - // Mark evicted - cleanup will happen immediately if no tasks are using it, - // or when the last task releases it. + // Mark evicted. The session stays in the authoritative sessions map until cleanup. + // If refCount > 0, cleanup is deferred until all tasks release. state.markEvicted() } }) @@ -625,18 +751,48 @@ private[spark] class Executor( (accums, accUpdates) } + /** + * Obtains an IsolatedSessionState for the given job artifact state. + * Gets or creates a session from the cache, then acquires it. We need to retry the cache + * lookup if the session was evicted between get() and acquire(). This can happen when the + * cache is full and another task triggers eviction. + */ + private def obtainSession(jobArtifactState: JobArtifactState): IsolatedSessionState = { + var session: IsolatedSessionState = null + var acquired = false + while (!acquired) { + // Get or create session. The loader uses sessions map as the authoritative store. + // This ensures there's only one IsolatedSessionState per UUID at any time. + session = isolatedSessionCache.get(jobArtifactState.uuid, () => { + // Check the authoritative sessions map first. tryUnEvict() will block if + // cleanup is in progress, so when it returns false, the session is already + // removed from the map and it's safe to create a new one. + val existingSession = IsolatedSessionState.sessions.get(jobArtifactState.uuid) + if (existingSession != null && existingSession.tryUnEvict()) { + existingSession + } else { + newSessionState(jobArtifactState) + } + }) + // acquire() can return false if session was evicted between get() and now. + // In that case, retry - the session is already removed from cache. + acquired = session.acquire() + } + session + } + override def run(): Unit = { // Classloader isolation val isolatedSession = taskDescription.artifacts.state match { case Some(jobArtifactState) => - isolatedSessionCache.get(jobArtifactState.uuid, () => newSessionState(jobArtifactState)) - case _ => defaultSessionState + obtainSession(jobArtifactState) + case _ => + // The default session is never in the cache and never evicted, + // so no need to acquire/release. + defaultSessionState } - // Pin the session to prevent its class loader from being closed while this task is running. - isolatedSession.acquire() - setMDCForTask(taskName, mdcProperties) threadId = Thread.currentThread.getId Thread.currentThread.setName(threadName) @@ -943,7 +1099,10 @@ private[spark] class Executor( metricsPoller.onTaskCompletion(taskId, task.stageId, task.stageAttemptId) } // Release the session reference. If evicted and this was the last task, cleanup happens. - isolatedSession.release() + // Skip for defaultSessionState since it's never evicted. + if (isolatedSession ne defaultSessionState) { + isolatedSession.release() + } } } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSideSessionManagementSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSideSessionManagementSuite.scala new file mode 100644 index 0000000000000..f127951054e78 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSideSessionManagementSuite.scala @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import java.io.File + +import scala.collection.mutable.HashMap + +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfterEach +import org.scalatestplus.mockito.MockitoSugar + +import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} +import org.apache.spark.util.{MutableURLClassLoader, Utils} + +/** + * Unit tests for IsolatedSessionState lifecycle management. + * These tests verify the fix for race conditions in session acquire/release/eviction. + */ +class ExecutorSideSessionManagementSuite + extends SparkFunSuite + with BeforeAndAfterEach + with MockitoSugar { + + private var testSessionCounter = 0 + private var tempDir: File = _ + + override def beforeEach(): Unit = { + super.beforeEach() + // Clear the sessions map before each test + IsolatedSessionState.sessions.clear() + testSessionCounter = 0 + + // Set up a mock SparkEnv so that cleanup() can access SparkFiles.getRootDirectory() + tempDir = Utils.createTempDir() + val mockEnv = mock[SparkEnv] + val conf = new SparkConf(false) + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.driverTmpDir).thenReturn(Some(tempDir.getAbsolutePath)) + SparkEnv.set(mockEnv) + } + + override def afterEach(): Unit = { + // Clear the sessions map after each test + IsolatedSessionState.sessions.clear() + SparkEnv.set(null) + if (tempDir != null && tempDir.exists()) { + Utils.deleteRecursively(tempDir) + tempDir = null + } + super.afterEach() + } + + /** + * Creates a test IsolatedSessionState with a mock classloader and unique UUID. + */ + private def createTestSession(uuid: String): IsolatedSessionState = { + val classLoader = new MutableURLClassLoader( + Array.empty, + Thread.currentThread().getContextClassLoader + ) + val session = new IsolatedSessionState( + sessionUUID = uuid, + urlClassLoader = classLoader, + replClassLoader = classLoader, + currentFiles = new HashMap[String, Long](), + currentJars = new HashMap[String, Long](), + currentArchives = new HashMap[String, Long](), + replClassDirUri = None + ) + // Register in authoritative sessions map as would happen in production + IsolatedSessionState.sessions.put(uuid, session) + session + } + + private def nextUniqueUuid(): String = { + testSessionCounter += 1 + s"test-uuid-$testSessionCounter" + } + + test("acquire returns true for new session") { + val session = createTestSession(nextUniqueUuid()) + assert(session.acquire()) + } + + test("acquire returns true for session acquired multiple times") { + val session = createTestSession(nextUniqueUuid()) + assert(session.acquire()) + assert(session.acquire()) + assert(session.acquire()) + } + + test("acquire returns false after session is evicted with no references") { + val session = createTestSession(nextUniqueUuid()) + session.markEvicted() + // Session should be cleaned up immediately since refCount is 0 + assert(!IsolatedSessionState.sessions.containsKey(session.sessionUUID)) + // Cannot acquire an evicted session. + assert(!session.acquire()) + } + + test("acquire returns false after session is evicted even with existing references") { + val uuid = nextUniqueUuid() + val session = createTestSession(uuid) + + // First task acquires the session + assert(session.acquire()) + + // Session gets evicted (e.g., due to cache pressure) + session.markEvicted() + // Session should still be in the map because refCount > 0 (deferred cleanup) + assert(IsolatedSessionState.sessions.containsKey(uuid)) + + // A new task tries to acquire the same session - should fail because it's evicted + assert(!session.acquire()) + + // The original task releases - now cleanup happens + session.release() + assert(!IsolatedSessionState.sessions.containsKey(uuid)) + } + + test("deferred cleanup with multiple references") { + val uuid = nextUniqueUuid() + val session = createTestSession(uuid) + + // Acquire the session multiple times (simulating multiple tasks) + assert(session.acquire()) + assert(session.acquire()) + assert(session.acquire()) + + // Evict the session - cleanup should be deferred + session.markEvicted() + assert(IsolatedSessionState.sessions.containsKey(uuid)) + + // Release twice - cleanup should still be deferred + session.release() + assert(IsolatedSessionState.sessions.containsKey(uuid)) + session.release() + assert(IsolatedSessionState.sessions.containsKey(uuid)) + + // Release the last reference - cleanup should happen now + session.release() + assert(!IsolatedSessionState.sessions.containsKey(uuid)) + } + + test("tryUnEvict succeeds when session is evicted but still has references") { + val session = createTestSession(nextUniqueUuid()) + + // Acquire the session + assert(session.acquire()) + + // Evict the session + session.markEvicted() + + // Try to un-evict - should succeed because refCount > 0 + assert(session.tryUnEvict()) + + // Now acquire should succeed again + assert(session.acquire()) + } + + test("tryUnEvict fails when session is not evicted") { + val session = createTestSession(nextUniqueUuid()) + + // Acquire without eviction + assert(session.acquire()) + + // Try to un-evict - should fail because session is not evicted + assert(!session.tryUnEvict()) + } + + test("tryUnEvict and acquire fail when session has no references") { + val uuid = nextUniqueUuid() + val session = createTestSession(uuid) + + // Evict with no references - triggers immediate cleanup + session.markEvicted() + assert(!IsolatedSessionState.sessions.containsKey(uuid)) + + // tryUnEvict should fail because refCount is 0 and session is already cleaned up + assert(!session.tryUnEvict()) + + // acquire should also fail + assert(!session.acquire()) + } + + test("session reuse via tryUnEvict keeps session in map when not evicted") { + // Note: This test verifies `IsolatedSessionState.sessions` behavior in isolation. + // In production, after tryUnEvict(), the session is put back into the Guava cache. + // When the cache eventually evicts it again (due to LRU policy), markEvicted() will be called, + // and cleanup will happen if refCount is 0. So there's no resource leak in practice. + val uuid = nextUniqueUuid() + val session = createTestSession(uuid) + + // Simulate task 1 acquiring the session + assert(session.acquire()) + + // Session gets evicted (e.g., due to cache pressure) + session.markEvicted() + assert(IsolatedSessionState.sessions.containsKey(uuid)) // Deferred cleanup + + // Simulate cache loader trying to reuse the session via tryUnEvict + assert(session.tryUnEvict()) + + // Now a new task can acquire the session + assert(session.acquire()) + + // Task 1 releases + session.release() + assert(IsolatedSessionState.sessions.containsKey(uuid)) // Still has 1 reference + + // Task 2 releases - session stays in map because it's not evicted + session.release() + // Session stays in map because it's not evicted anymore (was un-evicted). + // In production, the Guava cache would eventually evict it again, triggering cleanup. + assert(IsolatedSessionState.sessions.containsKey(uuid)) + } +} diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala index 535c0d2180d40..8bd6c5cf01681 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/RemoteSparkSession.scala @@ -134,7 +134,7 @@ object SparkConnectServerUtils { // Testing SPARK-49673, setting maxBatchSize to 10MiB s"spark.connect.grpc.arrow.maxBatchSize=${10 * 1024 * 1024}", // Cache less sessions to save memory. - "spark.executor.isolatedSessionCache.size=5", + "spark.executor.isolatedSessionCache.size=10", // Disable UI "spark.ui.enabled=false").flatMap(v => "--conf" :: v :: Nil) } From 20974ced5a78d0771b07405ac3e3ad17aa267b44 Mon Sep 17 00:00:00 2001 From: Emilie Faracci Date: Sat, 31 Jan 2026 16:52:34 +0900 Subject: [PATCH 371/400] [SPARK-55290][NETWORK][TESTS] Fix testReloadMissingTrustStore cross-device link error with JDK 21 ### What changes were proposed in this pull request? Replace hardcoded filename with `File.createTempFile()` in `testReloadMissingTrustStore` to ensure the temporary `trustStore` file is created in the same filesystem as the test working directory. ### Why are the changes needed? This change fixes a unit test bug. JDK 21's `UnixFileSystem.move()` uses atomic rename operations that fail when moving files across different mounted filesystems. The test creates a temporary file in `/tmp` and attempts to move it to the current directory. When these locations are on different partitions, the atomic move fails with a `testmissing.jks: Cross-device link` error because the `rename()` system call cannot move files across different filesystems. The fix ensures both temporary and target files are created in the same filesystem by eliminating the cross-filesystem move operation. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Verified `ReloadingX509TrustManagerSuite` passes with both JDK 21 and JDK 17: `build/mvn -pl common/network-common -Dtest=ReloadingX509TrustManagerSuite test` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54073 from efaracci018/fix-testReloadMissingTrustStore. Authored-by: Emilie Faracci Signed-off-by: Kousuke Saruta (cherry picked from commit 866a6e8af0a0521f14874eddf05f601b3a1f7880) Signed-off-by: Kousuke Saruta --- .../spark/network/ssl/ReloadingX509TrustManagerSuite.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/ssl/ReloadingX509TrustManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ssl/ReloadingX509TrustManagerSuite.java index 5bb47ff388671..b373d99a8e404 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ssl/ReloadingX509TrustManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ssl/ReloadingX509TrustManagerSuite.java @@ -189,7 +189,9 @@ public void testReload() throws Exception { public void testReloadMissingTrustStore() throws Exception { KeyPair kp = generateKeyPair("RSA"); X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); - File trustStore = new File("testmissing.jks"); + File trustStore = File.createTempFile("testmissing", "jks"); + trustStore.delete(); + // trustStore is going to be re-created later so delete it on exit. trustStore.deleteOnExit(); assertFalse(trustStore.exists()); createTrustStore(trustStore, "password", "cert1", cert1); From e3de8690059bd75c40e59a03371b244ff966b779 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Tue, 3 Feb 2026 02:13:07 -0800 Subject: [PATCH 372/400] [SPARK-54812][SQL][4.1] Make executable commands not execute on resultDf.cache() ### What changes were proposed in this pull request? Backport of #53572 to 4.1 branch Follow up of https://github.com/apache/spark/pull/51032 . That pr changed V2WriteCommand not to execute eagerly on df.cache(). However, there are a bunch of other commands that still do. ``` val df = sql("CREATE TABLE...") df.cache() // executes again, fails with TableAlreadyExistsException ``` This patch skip CacheManager for all Command, because these are eagerly-executed already when first calling sql("COMMAND"). ``` val df = sql("SHOW TABLES.") sql("CREATE TABLE foo") df.cache() // executes again and df now includes foo ``` ### Why are the changes needed? To prevent the command with side-effect from being executed again if a user runs df.cache on the result of the command. Many are dangerous as they would be running a second time without the user expectation (df.cache triggering another action on the table) ### Does this PR introduce _any_ user-facing change? If the user created a resultDF from a command, and then ran resultDf.cache, it used to re-run the command. Now it will no-op. Most of the time, this is beneficial as re-running the command will result in an error, or worse data corruption. However, in some small cases , like SHOW TABLES or SHOW NAMESPACES, it will affect the contents of resultDf as it will no longer refresh when calling resultDf.cache() Note: In most cases, we are lucky and will not see user-facing change. This is because commands, like for example DescribeTableExec plan node, already has a in-memory reference to Table object and keeps the old result despite repeated execution. However, SHOW XXX command plans do not cache in memory results so they see some effect. ### How was this patch tested? Existing unit test, add new unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #54064 from szehon-ho/4.1. Authored-by: Szehon Ho Signed-off-by: Dongjoon Hyun --- .../plans/logical/IgnoreCachedData.scala | 23 ----- .../catalyst/plans/logical/v2Commands.scala | 3 +- .../spark/sql/connect/CatalogSuite.scala | 65 ++++++++++++++ .../spark/sql/execution/CacheManager.scala | 6 +- .../CreateUserDefinedFunctionCommand.scala | 3 +- .../sql/execution/command/SetCommand.scala | 3 +- .../spark/sql/execution/command/cache.scala | 3 +- .../apache/spark/sql/DatasetCacheSuite.scala | 29 ++++++ .../DataSourceV2DataFrameSuite.scala | 88 +++++++++++++++++++ 9 files changed, 189 insertions(+), 34 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala deleted file mode 100644 index 85958cb43d4f8..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.plans.logical - -/** - * A [[LogicalPlan]] operator that does not use the cached results stored in CacheManager - */ -trait IgnoreCachedData extends LogicalPlan {} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 17c5eb1593bfe..3f5a006c505ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -61,8 +61,7 @@ trait KeepAnalyzedQuery extends Command { trait V2WriteCommand extends UnaryCommand with KeepAnalyzedQuery - with CTEInChildren - with IgnoreCachedData { + with CTEInChildren { def table: NamedRelation def query: LogicalPlan def isByName: Boolean diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala index bae569978890b..40b5444b84ed3 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala @@ -300,4 +300,69 @@ class CatalogSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelpe } } } + + test("SPARK-52312: DESCRIBE TABLE result should not change after ALTER TABLE") { + val tableName = "test_describe_cache" + withTable(tableName) { + spark.sql(s"CREATE TABLE $tableName (c1 int, c2 string) USING parquet") + + // Describe the table and cache the result + val describeDf = spark.sql(s"DESCRIBE TABLE $tableName") + describeDf.cache() + + // Verify initial describe shows original columns + val originalColumns = describeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(originalColumns.contains("c1")) + assert(originalColumns.contains("c2")) + assert(!originalColumns.contains("c3")) + + // Alter the table to add a new column + spark.sql(s"ALTER TABLE $tableName ADD COLUMN c3 double") + + // The describe result df should still show the original schema (before ALTER) + val cachedColumns = describeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(cachedColumns.contains("c1")) + assert(cachedColumns.contains("c2")) + assert( + !cachedColumns.contains("c3"), + "DESCRIBE column df should not reflect the new column") + + // A fresh DESCRIBE TABLE call should show the new schema (with c3) + // because Describe command is not cached + val freshDescribeDf = spark.sql(s"DESCRIBE TABLE $tableName") + val freshColumns = freshDescribeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(freshColumns.contains("c1")) + assert(freshColumns.contains("c2")) + assert(freshColumns.contains("c3"), "Fresh DESCRIBE should reflect the new column") + } + } + + test("SPARK-52312: DESCRIBE TABLE cache should be a no-op") { + val tableName = "test_describe_cache_noop" + withTable(tableName) { + spark.sql(s"CREATE TABLE $tableName (c1 int, c2 string) USING parquet") + + // Create describe DataFrame but don't cache yet + val describeDf = spark.sql(s"DESCRIBE TABLE $tableName") + + // add column c3 + spark.sql(s"ALTER TABLE $tableName ADD COLUMN c3 double") + + // Now cache the describe + describeDf.cache() + + // Verify describe shows schema at the initialization of describeDf + val cachedColumns = describeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(cachedColumns.contains("c1")) + assert(cachedColumns.contains("c2")) + assert(!cachedColumns.contains("c3"), "DESCRIBE should not see c3 added before cache") + + // A fresh DESCRIBE TABLE call should show the latest schema (with c3) + val freshDescribeDf = spark.sql(s"DESCRIBE TABLE $tableName") + val freshColumns = freshDescribeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(freshColumns.contains("c1")) + assert(freshColumns.contains("c2")) + assert(freshColumns.contains("c3")) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 34e47084f6561..a551762b83899 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.V2TableReference import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint -import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint, View} +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, ResolvedHint, View} import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.classic.{Dataset, SparkSession} @@ -138,7 +138,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { storageLevel: StorageLevel): Unit = { if (storageLevel == StorageLevel.NONE) { // Do nothing for StorageLevel.NONE since it will not actually cache any data. - } else if (unnormalizedPlan.isInstanceOf[IgnoreCachedData]) { + } else if (unnormalizedPlan.isInstanceOf[Command]) { logWarning( log"Asked to cache a plan that is inapplicable for caching: " + log"${MDC(LOGICAL_PLAN, unnormalizedPlan)}" @@ -499,7 +499,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { */ private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = { val newPlan = plan transformDown { - case command: IgnoreCachedData => command + case command: Command => command case currentFragment => lookupCachedDataInternal(currentFragment).map { cached => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala index 78ff514bf9e51..a3780a8bff197 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala @@ -22,7 +22,6 @@ import java.util.Locale import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CapturesConfig, FunctionIdentifier} import org.apache.spark.sql.catalyst.catalog.{LanguageSQL, RoutineLanguage, UserDefinedFunctionErrors} -import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -30,7 +29,7 @@ import org.apache.spark.sql.types.StructType * The base class for CreateUserDefinedFunctionCommand */ abstract class CreateUserDefinedFunctionCommand - extends LeafRunnableCommand with IgnoreCachedData with CapturesConfig + extends LeafRunnableCommand with CapturesConfig object CreateUserDefinedFunctionCommand { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index e31e7e8d704ca..e248f0eea96de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.VariableResolution import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.errors.QueryCompilationErrors.toSQLId @@ -218,7 +217,7 @@ object SetCommand { * reset spark.sql.session.timeZone; * }}} */ -case class ResetCommand(config: Option[String]) extends LeafRunnableCommand with IgnoreCachedData { +case class ResetCommand(config: Option[String]) extends LeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val globalInitialConfigs = sparkSession.sharedState.conf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index de5dbddbfa146..9c3ac9ef74191 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -18,12 +18,11 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData /** * Clear all cached data from the in-memory cache. */ -case object ClearCacheCommand extends LeafRunnableCommand with IgnoreCachedData { +case object ClearCacheCommand extends LeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.catalog.clearCache() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 510eeb4fa00bd..627811eaecf8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -337,4 +337,33 @@ class DatasetCacheSuite extends QueryTest val value = ColumnarToRowExec(plan).executeCollectPublic().head.get(0) assert(value == LocalTime.of(13, 33, 33)) } + + test("SPARK-54812: SHOW TABLES should be a no-op.") { + val t1 = "show_tables_test_t1" + val t2 = "show_tables_test_t2" + withTable(t1, t2) { + // Create initial table + sql(s"CREATE TABLE $t1 (c1 int) USING parquet") + + // Run SHOW TABLES and save to a DataFrame + val showTablesDf = sql("SHOW TABLES") + + // Add another table after creating the DataFrame + sql(s"CREATE TABLE $t2 (c1 int) USING parquet") + + // Cache the DataFrame - this should reflect the latest state + showTablesDf.cache() + + // Verify cached result reflects the latest state (includes t2) + val cachedTables = showTablesDf.select("tableName").collect().map(_.getString(0)).toSet + assert(cachedTables.contains(t1)) + assert(!cachedTables.contains(t2)) + + // A fresh SHOW TABLES call should also show both tables + val freshShowTablesDf = sql("SHOW TABLES") + val freshTables = freshShowTablesDf.select("tableName").collect().map(_.getString(0)).toSet + assert(freshTables.contains(t1)) + assert(freshTables.contains(t2)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index c3164b3428f95..b2565d22e2178 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -2047,6 +2047,94 @@ class DataSourceV2DataFrameSuite } } + test("SPARK-54812: caching dataframe created from CREATE shouldn't re-execute the command") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + val df = sql(s"CREATE TABLE $t USING foo AS SELECT 1 AS c1, 'a' AS c2") + + // Verify the table was created with the correct data + checkAnswer(spark.table(t), Row(1, "a")) + + // Caching the DataFrame created from CREATE TABLE AS SELECT should not re-execute + // the command. If it did, it would fail with TableAlreadyExistsException. + df.cache() + + // The cached result should be empty (CTAS returns no rows) + checkAnswer(df, Seq.empty) + } + } + + test("SPARK-54812: caching dataframe created from ALTER TABLE shouldn't re-execute the command") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (c1 int) USING foo") + sql(s"INSERT INTO $t VALUES (1), (2)") + + // Add a column via ALTER TABLE + val alterDf = sql(s"ALTER TABLE $t ADD COLUMN c2 string") + + // Verify the column was added + assert(spark.table(t).schema.fieldNames.toSeq == Seq("c1", "c2")) + + // Caching the DataFrame created from ALTER TABLE should not re-execute the command. + // If it did, it would fail because the column already exists. + alterDf.cache() + + // Schema should still have the same columns (not duplicated) + assert(spark.table(t).schema.fieldNames.toSeq == Seq("c1", "c2")) + + // The cached result should be empty (ALTER TABLE returns no rows) + checkAnswer(alterDf, Seq.empty) + } + } + + test("SPARK-54812: caching dataframe created from DROP TABLE shouldn't re-execute the command") { + val t = "testcat.ns1.ns2.tbl" + sql(s"CREATE TABLE $t (c1 int, c2 string) USING foo") + sql(s"INSERT INTO $t VALUES (1, 'a'), (2, 'b')") + + // Drop the table + val dropDf = sql(s"DROP TABLE $t") + + // Verify the table no longer exists + assert(!spark.catalog.tableExists(t)) + + // Caching the DataFrame created from DROP TABLE should not re-execute the command. + // If it did, it would fail with NoSuchTableException. + dropDf.cache() + + // The cached result should be empty (DROP TABLE returns no rows) + checkAnswer(dropDf, Seq.empty) + } + + test("SPARK-54812: DESCRIBE TABLE v2 cache should be a no-op") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (c1 int, c2 string) USING foo") + + // Create describe DataFrame but don't cache yet + val describeDf = sql(s"DESCRIBE TABLE $t") + + // add column c3 + sql(s"ALTER TABLE $t ADD COLUMN c3 double") + + describeDf.cache() + + // Verify describe shows schema at the initialization of describeDf + val cachedColumns = describeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(cachedColumns.contains("c1")) + assert(cachedColumns.contains("c2")) + assert(!cachedColumns.contains("c3"), "Cached DESCRIBE should reflect c3 added before cache") + + // A fresh DESCRIBE TABLE call should show the latest schema (with c3) + val freshDescribeDf = sql(s"DESCRIBE TABLE $t") + val freshColumns = freshDescribeDf.select("col_name").collect().map(_.getString(0)).toSet + assert(freshColumns.contains("c1")) + assert(freshColumns.contains("c2")) + assert(freshColumns.contains("c3")) + } + } + private def pinTable(catalogName: String, ident: Identifier, version: String): Unit = { catalog(catalogName) match { case inMemory: BasicInMemoryTableCatalog => inMemory.pinTable(ident, version) From 0b8a641a2e067db2d0025fea24678e336ceea727 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 3 Feb 2026 08:05:44 -0800 Subject: [PATCH 373/400] [SPARK-55258][DOCS] Document CLI parameters in declarative pipelines programming guide ### What changes were proposed in this pull request? Documents parameters for the `spark-pipelines` CLI, in the declarative pipelines programming guide ### Why are the changes needed? Complete documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? Closes #54035 from sryza/refresh-selection-docs. Authored-by: Sandy Ryza Signed-off-by: Sandy Ryza (cherry picked from commit 9788c52426df29fe4d145255f7a7f945bee96d3a) Signed-off-by: Sandy Ryza --- ...declarative-pipelines-programming-guide.md | 47 +++++++++++++++++-- 1 file changed, 44 insertions(+), 3 deletions(-) diff --git a/docs/declarative-pipelines-programming-guide.md b/docs/declarative-pipelines-programming-guide.md index 5b3a06fe26c0e..c5d18a7cb71be 100644 --- a/docs/declarative-pipelines-programming-guide.md +++ b/docs/declarative-pipelines-programming-guide.md @@ -117,10 +117,47 @@ The `spark-pipelines` command line interface (CLI) is the primary way to manage `spark-pipelines run` launches an execution of a pipeline and monitors its progress until it completes. -The `--spec` parameter allows selecting the pipeline spec file. If not provided, the CLI will look in the current directory and parent directories for one of the files: +Since `spark-pipelines` is built on top of `spark-submit`, it supports all `spark-submit` arguments except for `--class`. For the complete list of available parameters, see the [Spark Submit documentation](https://spark.apache.org/docs/latest/submitting-applications.html#launching-applications-with-spark-submit). -* `spark-pipeline.yml` -* `spark-pipeline.yaml` +It also supports several pipeline-specific parameters: + +* `--spec PATH` - Path to the pipeline specification file. If not provided, the CLI will look in the current directory and parent directories for one of the files: + * `spark-pipeline.yml` + * `spark-pipeline.yaml` + +* `--full-refresh DATASETS` - List of datasets to reset and recompute (comma-separated). This clears all existing data and checkpoints for the specified datasets and recomputes them from scratch. + +* `--full-refresh-all` - Perform a full graph reset and recompute. This is equivalent to `--full-refresh` for all datasets in the pipeline. + +* `--refresh DATASETS` - List of datasets to update (comma-separated). This triggers an update for the specified datasets without clearing existing data. + +#### Refresh Selection Behavior + +If no refresh options are specified, a default incremental update is performed. The refresh parameters are mutually exclusive: +- `--full-refresh-all` cannot be combined with `--full-refresh` or `--refresh` +- `--full-refresh` and `--refresh` can be used together to specify different behaviors for different datasets + +#### Examples + +```bash +# Basic run with default incremental update +spark-pipelines run + +# Run with specific spec file +spark-pipelines run --spec /path/to/my-pipeline.yaml + +# Full refresh of specific datasets +spark-pipelines run --full-refresh orders,customers + +# Full refresh of entire pipeline +spark-pipelines run --full-refresh-all + +# Run with custom Spark configuration +spark-pipelines run --conf spark.sql.shuffle.partitions=200 --driver-memory 4g + +# Run on remote Spark Connect server +spark-pipelines run --remote sc://my-cluster:15002 +``` ### `spark-pipelines dry-run` @@ -129,6 +166,10 @@ The `--spec` parameter allows selecting the pipeline spec file. If not provided, - Analysis errors – e.g. selecting from a table or a column that doesn't exist - Graph validation errors - e.g. cyclic dependencies +Since `spark-pipelines` is built on top of `spark-submit`, it supports all `spark-submit` arguments except for `--class`. For the complete list of available parameters, see the [Spark Submit documentation](https://spark.apache.org/docs/latest/submitting-applications.html#launching-applications-with-spark-submit). + +It also supports the pipeline-specific `--spec` parameter (see description above in the `run` section). + ## Programming with SDP in Python SDP Python definitions are defined in the `pyspark.pipelines` module. From bc3559d0e623f60e745f498cc5125ffbce380f94 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 8 Feb 2026 00:12:50 +0800 Subject: [PATCH 374/400] [SPARK-55387][CORE][UI] Fix DAG visualization not rendering due to malformed DOT label ### What changes were proposed in this pull request? This PR fixes a typo in `RDDOperationGraph.scala` where an extra `}` character was accidentally added to the DOT node label string, causing the DAG visualization to fail rendering in both Jobs and Stages pages of the Spark Web UI. **Before (broken):** ```scala s"""${node.id} [id="node_${node.id}" labelType="html" label="$label}"]""" ``` **After (fixed):** ```scala s"""${node.id} [id="node_${node.id}" labelType="html" label="$label"]""" ``` ### Why are the changes needed? The DAG visualization in the Spark Web UI is completely broken. Instead of rendering the graph, it shows the raw DOT source text. This affects both Jobs and Stages views across all browsers (Firefox, Edge, Chrome) and both Spark Shell and PySpark. This regression was introduced in SPARK-45274 (PR #43053). ### Does this PR introduce _any_ user-facing change? Yes. The DAG visualization in the Spark Web UI will render correctly again. **Before:** Raw DOT graph source displayed instead of visualization **After:** Properly rendered DAG graph ### How was this patch tested? - Existing unit tests pass: `RDDOperationGraphSuite`, `UISeleniumSuite` (DAG visualization test) - Manual verification with `sc.range(0, 10, 1, 4).count()` in spark-shell ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54171 from yaooqinn/SPARK-55387. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit 4e86ae41de180853246b8dccbc0fe55f70ef5428) Signed-off-by: Kent Yao --- .../scala/org/apache/spark/ui/scope/RDDOperationGraph.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index cd057ed08c3c3..4aa4954b84a91 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -269,7 +269,7 @@ private[spark] object RDDOperationGraph extends Logging { val label = StringEscapeUtils.escapeJava( s"${node.name} [${node.id}]$isCached$isBarrier$outputDeterministicLevel" + s"
$escapedCallsite") - s"""${node.id} [id="node_${node.id}" labelType="html" label="$label}"]""" + s"""${node.id} [id="node_${node.id}" labelType="html" label="$label"]""" } /** Update the dot representation of the RDDOperationGraph in cluster to subgraph. From 10b405d16d172c6a3a9d081fe98733fd58014835 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 7 Feb 2026 21:55:31 -0800 Subject: [PATCH 375/400] [SPARK-55423][INFRA] Set `strategy.max-parrallel` to 20 for all GitHub Action jobs ### What changes were proposed in this pull request? This PR aims to set `strategy.max-parrallel` to 20 for all GitHub Action jobs. ### Why are the changes needed? ASF Infra team directly requested us via email in (privatespark) mailing list. - https://lists.apache.org/thread/voqz9tp3m8wj00lp0y81n25qgvc90f3q Here is `GitHub Action` syntax. - https://docs.github.com/en/actions/reference/workflows-and-actions/workflow-syntax#jobsjob_idstrategymax-parallel Screenshot 2026-02-07 at 21 09 02 ### Does this PR introduce _any_ user-facing change? No Apache Spark behavior change. - Technically, for the PR builder, we use more 20 jobs on the PR contributor's GitHub repo. This job will be limited. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: `Opus 4.5` on `Claude Code` Closes #54204 from dongjoon-hyun/SPARK-55423. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit de345288830c257bcac4ef6cca7c1be3c031b4f5) Signed-off-by: Dongjoon Hyun --- .github/workflows/benchmark.yml | 1 + .github/workflows/build_and_test.yml | 2 ++ .github/workflows/maven_test.yml | 1 + .github/workflows/publish_snapshot.yml | 1 + .github/workflows/python_hosted_runner_test.yml | 1 + 5 files changed, 6 insertions(+) diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml index 3e90bb329be56..eb42160c378de 100644 --- a/.github/workflows/benchmark.yml +++ b/.github/workflows/benchmark.yml @@ -136,6 +136,7 @@ jobs: runs-on: ubuntu-latest strategy: fail-fast: false + max-parallel: 20 matrix: split: ${{fromJSON(needs.matrix-gen.outputs.matrix)}} env: diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 5735dc6ad2088..742990622712b 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -231,6 +231,7 @@ jobs: timeout-minutes: 120 strategy: fail-fast: false + max-parallel: 20 matrix: java: - ${{ inputs.java }} @@ -503,6 +504,7 @@ jobs: image: ${{ needs.precondition.outputs.image_pyspark_url_link }} strategy: fail-fast: false + max-parallel: 20 matrix: java: - ${{ inputs.java }} diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 7828c4bdc6ed9..580593f1cfe5e 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -59,6 +59,7 @@ jobs: timeout-minutes: 150 strategy: fail-fast: false + max-parallel: 20 matrix: java: - ${{ inputs.java }} diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index b20c34d8ec5a1..0608ba1fd1f1b 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -36,6 +36,7 @@ jobs: runs-on: ubuntu-latest strategy: fail-fast: false + max-parallel: 20 matrix: # keep in sync with default value of workflow_dispatch input 'branch' branch: ${{ fromJSON( inputs.branch || '["master", "branch-4.0", "branch-3.5"]' ) }} diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml index c0fe016326d49..659171b901d3f 100644 --- a/.github/workflows/python_hosted_runner_test.yml +++ b/.github/workflows/python_hosted_runner_test.yml @@ -62,6 +62,7 @@ jobs: timeout-minutes: 120 strategy: fail-fast: false + max-parallel: 20 matrix: java: - ${{ inputs.java }} From a579c87f8220d192c81a6b0086b7b7487bab9e8d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 8 Feb 2026 21:09:30 +0500 Subject: [PATCH 376/400] [SPARK-55337][SS] Fix MemoryStream backward compatibility This is a followup to #52402 that addresses backward compatibility concerns: 1. Keep the original `implicit SQLContext` factory methods for full backward compatibility 2. Add new overloads with explicit `SparkSession` parameter for new code 3. Fix `TestGraphRegistrationContext` to provide implicit `spark` and `sqlContext` to avoid name shadowing issues in nested classes 4. Remove redundant `implicit val sparkSession` declarations from pipeline tests that are no longer needed with the fix PR #52402 changed the MemoryStream API to use `implicit SparkSession` which broke backward compatibility for code that only has `implicit SQLContext` available. This followup ensures: - Old code continues to work without modification - New code can use SparkSession with explicit parameters - Internal implementation uses SparkSession (modernization from #52402) No. This maintains full backward compatibility while adding new API options. Existing tests pass. The API changes are additive. Yes Made with [Cursor](https://cursor.com) Closes #54108 from cloud-fan/memory-stream-compat. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit db28b99ded98dd2258d2e3a6d13f9b366cc0ad3d) Signed-off-by: Wenchen Fan --- .../execution/streaming/runtime/memory.scala | 57 +++++++++----- .../sources/ContinuousMemoryStream.scala | 51 +++++------- .../sources/LowLatencyMemoryStream.scala | 56 +++++-------- .../PythonStreamingDataSourceSuite.scala | 4 +- .../execution/streaming/MemorySinkSuite.scala | 78 ------------------- .../state/StateStoreCoordinatorSuite.scala | 34 ++++---- .../streaming/state/StateStoreSuite.scala | 4 +- .../FlatMapGroupsWithStateSuite.scala | 2 +- .../streaming/StreamingAggregationSuite.scala | 2 +- .../StreamingDeduplicationSuite.scala | 2 +- .../sql/streaming/StreamingJoinSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- .../graph/ConnectInvalidPipelineSuite.scala | 5 +- .../graph/ConnectValidPipelineSuite.scala | 6 -- .../graph/MaterializeTablesSuite.scala | 8 +- .../pipelines/graph/SystemMetadataSuite.scala | 5 -- .../graph/TriggeredGraphExecutionSuite.scala | 6 +- .../utils/TestGraphRegistrationContext.scala | 13 +++- 18 files changed, 117 insertions(+), 220 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala index bf67ed670ec81..c7556ed478599 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala @@ -43,36 +43,51 @@ import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -object MemoryStream extends LowPriorityMemoryStreamImplicits { +object MemoryStream { protected val currentBlockId = new AtomicInteger(0) protected val memoryStreamId = new AtomicInteger(0) - def apply[A : Encoder](implicit sparkSession: SparkSession): MemoryStream[A] = - new MemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession) - - def apply[A : Encoder](numPartitions: Int)(implicit sparkSession: SparkSession): MemoryStream[A] = - new MemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, Some(numPartitions)) -} - -/** - * Provides lower-priority implicits for MemoryStream to prevent ambiguity when both - * SparkSession and SQLContext are in scope. The implicits in the companion object, - * which use SparkSession, take higher precedence. - */ -trait LowPriorityMemoryStreamImplicits { - this: MemoryStream.type => - - // Deprecated: Used when an implicit SQLContext is in scope - @deprecated("Use MemoryStream.apply with an implicit SparkSession instead of SQLContext", "4.1.0") - def apply[A: Encoder]()(implicit sqlContext: SQLContext): MemoryStream[A] = + /** + * Creates a MemoryStream with an implicit SQLContext (backward compatible). + * Usage: `MemoryStream[Int]` + */ + def apply[A: Encoder](implicit sqlContext: SQLContext): MemoryStream[A] = new MemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext.sparkSession) - @deprecated("Use MemoryStream.apply with an implicit SparkSession instead of SQLContext", "4.1.0") - def apply[A: Encoder](numPartitions: Int)(implicit sqlContext: SQLContext): MemoryStream[A] = + /** + * Creates a MemoryStream with specified partitions using implicit SQLContext. + * Usage: `MemoryStream[Int](numPartitions)` + */ + def apply[A: Encoder](numPartitions: Int)( + implicit sqlContext: SQLContext): MemoryStream[A] = new MemoryStream[A]( memoryStreamId.getAndIncrement(), sqlContext.sparkSession, Some(numPartitions)) + + /** + * Creates a MemoryStream with explicit SparkSession. + * Usage: `MemoryStream[Int](spark)` + */ + def apply[A: Encoder](sparkSession: SparkSession): MemoryStream[A] = + new MemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession) + + /** + * Creates a MemoryStream with specified partitions using explicit SparkSession. + * Usage: `MemoryStream[Int](spark, numPartitions)` + */ + def apply[A: Encoder](sparkSession: SparkSession, numPartitions: Int): MemoryStream[A] = + new MemoryStream[A]( + memoryStreamId.getAndIncrement(), + sparkSession, + Some(numPartitions)) + + /** + * Creates a MemoryStream with explicit encoder and SparkSession. + * Usage: `MemoryStream(Encoders.scalaInt, spark)` + */ + def apply[A](encoder: Encoder[A], sparkSession: SparkSession): MemoryStream[A] = + new MemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession)(encoder) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala index 8042cacf1374b..885f9ada22c9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala @@ -112,47 +112,36 @@ class ContinuousMemoryStream[A : Encoder]( override def commit(end: Offset): Unit = {} } -object ContinuousMemoryStream extends LowPriorityContinuousMemoryStreamImplicits { +object ContinuousMemoryStream { protected val memoryStreamId = new AtomicInteger(0) - def apply[A : Encoder](implicit sparkSession: SparkSession): ContinuousMemoryStream[A] = - new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession) - - def apply[A : Encoder](numPartitions: Int)(implicit sparkSession: SparkSession): - ContinuousMemoryStream[A] = - new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, numPartitions) - - def singlePartition[A : Encoder](implicit sparkSession: SparkSession): ContinuousMemoryStream[A] = - new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, 1) -} - -/** - * Provides lower-priority implicits for ContinuousMemoryStream to prevent ambiguity when both - * SparkSession and SQLContext are in scope. The implicits in the companion object, - * which use SparkSession, take higher precedence. - */ -trait LowPriorityContinuousMemoryStreamImplicits { - this: ContinuousMemoryStream.type => - - // Deprecated: Used when an implicit SQLContext is in scope - @deprecated("Use ContinuousMemoryStream with an implicit SparkSession " + - "instead of SQLContext", "4.1.0") - def apply[A: Encoder]()(implicit sqlContext: SQLContext): ContinuousMemoryStream[A] = + /** Creates a ContinuousMemoryStream with an implicit SQLContext (backward compatible). */ + def apply[A: Encoder](implicit sqlContext: SQLContext): ContinuousMemoryStream[A] = new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext.sparkSession) - @deprecated("Use ContinuousMemoryStream with an implicit SparkSession " + - "instead of SQLContext", "4.1.0") - def apply[A: Encoder](numPartitions: Int)(implicit sqlContext: SQLContext): - ContinuousMemoryStream[A] = + /** Creates a ContinuousMemoryStream with specified partitions (SQLContext). */ + def apply[A: Encoder](numPartitions: Int)( + implicit sqlContext: SQLContext): ContinuousMemoryStream[A] = new ContinuousMemoryStream[A]( memoryStreamId.getAndIncrement(), sqlContext.sparkSession, numPartitions) - @deprecated("Use ContinuousMemoryStream.singlePartition with an implicit SparkSession " + - "instead of SQLContext", "4.1.0") - def singlePartition[A: Encoder]()(implicit sqlContext: SQLContext): ContinuousMemoryStream[A] = + /** Creates a ContinuousMemoryStream with explicit SparkSession. */ + def apply[A: Encoder](sparkSession: SparkSession): ContinuousMemoryStream[A] = + new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession) + + /** Creates a ContinuousMemoryStream with specified partitions (SparkSession). */ + def apply[A: Encoder](sparkSession: SparkSession, numPartitions: Int): ContinuousMemoryStream[A] = + new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, numPartitions) + + /** Creates a single partition ContinuousMemoryStream (SQLContext). */ + def singlePartition[A: Encoder](implicit sqlContext: SQLContext): ContinuousMemoryStream[A] = new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext.sparkSession, 1) + + /** Creates a single partition ContinuousMemoryStream (SparkSession). */ + def singlePartition[A: Encoder](sparkSession: SparkSession): ContinuousMemoryStream[A] = + new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, 1) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/LowLatencyMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/LowLatencyMemoryStream.scala index d04f4b5d011ca..6dfeb0cc46032 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/LowLatencyMemoryStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/LowLatencyMemoryStream.scala @@ -172,53 +172,39 @@ class LowLatencyMemoryStream[A: Encoder]( } } -object LowLatencyMemoryStream extends LowPriorityLowLatencyMemoryStreamImplicits { +object LowLatencyMemoryStream { protected val memoryStreamId = new AtomicInteger(0) - def apply[A: Encoder](implicit sparkSession: SparkSession): LowLatencyMemoryStream[A] = - new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession) + /** Creates a LowLatencyMemoryStream with an implicit SQLContext (backward compatible). */ + def apply[A: Encoder](implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = + new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext.sparkSession) + /** Creates a LowLatencyMemoryStream with specified partitions (SQLContext). */ def apply[A: Encoder](numPartitions: Int)( - implicit - sparkSession: SparkSession): LowLatencyMemoryStream[A] = + implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = new LowLatencyMemoryStream[A]( memoryStreamId.getAndIncrement(), - sparkSession, - numPartitions = numPartitions - ) - - def singlePartition[A: Encoder](implicit sparkSession: SparkSession): LowLatencyMemoryStream[A] = - new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, 1) -} - -/** - * Provides lower-priority implicits for LowLatencyMemoryStream to prevent ambiguity when both - * SparkSession and SQLContext are in scope. The implicits in the companion object, - * which use SparkSession, take higher precedence. - */ -trait LowPriorityLowLatencyMemoryStreamImplicits { - this: LowLatencyMemoryStream.type => + sqlContext.sparkSession, + numPartitions = numPartitions) - // Deprecated: Used when an implicit SQLContext is in scope - @deprecated("Use LowLatencyMemoryStream with an implicit SparkSession " + - "instead of SQLContext", "4.1.0") - def apply[A: Encoder]()(implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = - new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext.sparkSession) + /** Creates a LowLatencyMemoryStream with explicit SparkSession. */ + def apply[A: Encoder](sparkSession: SparkSession): LowLatencyMemoryStream[A] = + new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession) - @deprecated("Use LowLatencyMemoryStream with an implicit SparkSession " + - "instead of SQLContext", "4.1.0") - def apply[A: Encoder](numPartitions: Int)(implicit sqlContext: SQLContext): - LowLatencyMemoryStream[A] = + /** Creates a LowLatencyMemoryStream with specified partitions (SparkSession). */ + def apply[A: Encoder](sparkSession: SparkSession, numPartitions: Int): LowLatencyMemoryStream[A] = new LowLatencyMemoryStream[A]( memoryStreamId.getAndIncrement(), - sqlContext.sparkSession, - numPartitions = numPartitions - ) + sparkSession, + numPartitions = numPartitions) - @deprecated("Use LowLatencyMemoryStream.singlePartition with an implicit SparkSession " + - "instead of SQLContext", "4.1.0") - def singlePartition[A: Encoder]()(implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = + /** Creates a single partition LowLatencyMemoryStream (SQLContext). */ + def singlePartition[A: Encoder](implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext.sparkSession, 1) + + /** Creates a single partition LowLatencyMemoryStream (SparkSession). */ + def singlePartition[A: Encoder](sparkSession: SparkSession): LowLatencyMemoryStream[A] = + new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sparkSession, 1) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala index 3b3e8687858dc..074a5cf9f2bf1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala @@ -859,7 +859,7 @@ class PythonStreamingDataSourceWriteSuite extends PythonDataSourceSuiteBase { val dataSource = createUserDefinedPythonDataSource(dataSourceName, simpleDataStreamWriterScript) spark.dataSource.registerPython(dataSourceName, dataSource) - val inputData = MemoryStream[Int](numPartitions = 3) + val inputData = MemoryStream[Int](spark, numPartitions = 3) val df = inputData.toDF() withTempDir { dir => val path = dir.getAbsolutePath @@ -943,7 +943,7 @@ class PythonStreamingDataSourceWriteSuite extends PythonDataSourceSuiteBase { |""".stripMargin val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript) spark.dataSource.registerPython(dataSourceName, dataSource) - val inputData = MemoryStream[Int](numPartitions = 3) + val inputData = MemoryStream[Int](spark, numPartitions = 3) withTempDir { dir => val path = dir.getAbsolutePath val checkpointDir = new File(path, "checkpoint") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala index e0ec3fd1b907b..4ec44eac22e36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala @@ -343,84 +343,6 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { intsToDF(expected)(schema)) } - test("LowPriorityMemoryStreamImplicits works with implicit sqlContext") { - // Test that MemoryStream can be created using implicit sqlContext - implicit val sqlContext: SQLContext = spark.sqlContext - - // Test MemoryStream[A]() with implicit sqlContext - val stream1 = MemoryStream[Int]() - assert(stream1 != null) - - // Test MemoryStream[A](numPartitions) with implicit sqlContext - val stream2 = MemoryStream[String](3) - assert(stream2 != null) - - // Verify the streams work correctly - stream1.addData(1, 2, 3) - val df1 = stream1.toDF() - assert(df1.schema.fieldNames.contains("value")) - - stream2.addData("a", "b", "c") - val df2 = stream2.toDF() - assert(df2.schema.fieldNames.contains("value")) - } - - test("LowPriorityContinuousMemoryStreamImplicits works with implicit sqlContext") { - import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream - // Test that ContinuousMemoryStream can be created using implicit sqlContext - implicit val sqlContext: SQLContext = spark.sqlContext - - // Test ContinuousMemoryStream[A]() with implicit sqlContext - val stream1 = ContinuousMemoryStream[Int]() - assert(stream1 != null) - - // Test ContinuousMemoryStream[A](numPartitions) with implicit sqlContext - val stream2 = ContinuousMemoryStream[String](3) - assert(stream2 != null) - - // Test ContinuousMemoryStream.singlePartition with implicit sqlContext - val stream3 = ContinuousMemoryStream.singlePartition[Int]() - assert(stream3 != null) - - // Verify the streams work correctly - stream1.addData(Seq(1, 2, 3)) - stream2.addData(Seq("a", "b", "c")) - stream3.addData(Seq(10, 20)) - - // Basic verification that streams are functional - assert(stream1.initialOffset() != null) - assert(stream2.initialOffset() != null) - assert(stream3.initialOffset() != null) - } - - test("LowPriorityLowLatencyMemoryStreamImplicits works with implicit sqlContext") { - import org.apache.spark.sql.execution.streaming.LowLatencyMemoryStream - // Test that LowLatencyMemoryStream can be created using implicit sqlContext - implicit val sqlContext: SQLContext = spark.sqlContext - - // Test LowLatencyMemoryStream[A]() with implicit sqlContext - val stream1 = LowLatencyMemoryStream[Int]() - assert(stream1 != null) - - // Test LowLatencyMemoryStream[A](numPartitions) with implicit sqlContext - val stream2 = LowLatencyMemoryStream[String](3) - assert(stream2 != null) - - // Test LowLatencyMemoryStream.singlePartition with implicit sqlContext - val stream3 = LowLatencyMemoryStream.singlePartition[Int]() - assert(stream3 != null) - - // Verify the streams work correctly - stream1.addData(Seq(1, 2, 3)) - stream2.addData(Seq("a", "b", "c")) - stream3.addData(Seq(10, 20)) - - // Basic verification that streams are functional - assert(stream1.initialOffset() != null) - assert(stream2.initialOffset() != null) - assert(stream3.initialOffset() != null) - } - private implicit def intsToDF(seq: Seq[Int])(implicit schema: StructType): DataFrame = { require(schema.fields.length === 1) sqlContext.createDataset(seq).toDF(schema.fieldNames.head) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala index 79bcdbca9ec69..4f2b78404131e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala @@ -123,14 +123,14 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { test("query stop deactivates related store providers") { var coordRef: StateStoreCoordinatorRef = null try { - implicit val spark: SparkSession = SparkSession.builder().sparkContext(sc).getOrCreate() + val spark: SparkSession = SparkSession.builder().sparkContext(sc).getOrCreate() SparkSession.setActiveSession(spark) import spark.implicits._ coordRef = spark.streams.stateStoreCoordinator spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "1") // Start a query and run a batch to load state stores - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val aggregated = inputData.toDF().groupBy("value").agg(count("*")) // stateful query val checkpointLocation = Utils.createTempDir().getAbsoluteFile val query = aggregated.writeStream @@ -253,8 +253,8 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { ) { case (coordRef, spark) => import spark.implicits._ - implicit val sparkSession: SparkSession = spark - val inputData = MemoryStream[Int] + + val inputData = MemoryStream[Int](spark) val query = setUpStatefulQuery(inputData, "query") // Add, commit, and wait multiple times to force snapshot versions and time difference (0 until 6).foreach { _ => @@ -289,10 +289,10 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { ) { case (coordRef, spark) => import spark.implicits._ - implicit val sparkSession: SparkSession = spark + // Start a join query and run some data to force snapshot uploads - val input1 = MemoryStream[Int] - val input2 = MemoryStream[Int] + val input1 = MemoryStream[Int](spark) + val input2 = MemoryStream[Int](spark) val df1 = input1.toDF().select($"value" as "leftKey", ($"value" * 2) as "leftValue") val df2 = input2.toDF().select($"value" as "rightKey", ($"value" * 3) as "rightValue") val joined = df1.join(df2, expr("leftKey = rightKey")) @@ -332,10 +332,10 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { ) { case (coordRef, spark) => import spark.implicits._ - implicit val sparkSession: SparkSession = spark + // Start and run two queries together with some data to force snapshot uploads - val input1 = MemoryStream[Int] - val input2 = MemoryStream[Int] + val input1 = MemoryStream[Int](spark) + val input2 = MemoryStream[Int](spark) val query1 = setUpStatefulQuery(input1, "query1") val query2 = setUpStatefulQuery(input2, "query2") @@ -399,9 +399,9 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { ) { case (coordRef, spark) => import spark.implicits._ - implicit val sparkSession: SparkSession = spark + // Start a query and run some data to force snapshot uploads - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val query = setUpStatefulQuery(inputData, "query") // Go through two batches to force two snapshot uploads. @@ -443,9 +443,9 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { ) { case (coordRef, spark) => import spark.implicits._ - implicit val sparkSession: SparkSession = spark + // Start a query and run some data to force snapshot uploads - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val query = setUpStatefulQuery(inputData, "query") // Go through several rounds of input to force snapshot uploads @@ -486,7 +486,7 @@ class StateStoreCoordinatorStreamingSuite extends StreamTest { SQLConf.STATE_STORE_COORDINATOR_SNAPSHOT_LAG_REPORT_INTERVAL.key -> "0" ) { withTempDir { srcDir => - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val query = inputData.toDF().dropDuplicates() val numPartitions = query.sparkSession.conf.get(SQLConf.SHUFFLE_PARTITIONS) // Keep track of state checkpoint directory for the second run @@ -608,7 +608,7 @@ class StateStoreCoordinatorStreamingSuite extends StreamTest { SQLConf.STATE_STORE_COORDINATOR_SNAPSHOT_LAG_REPORT_INTERVAL.key -> "0" ) { withTempDir { srcDir => - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val query = inputData.toDF().dropDuplicates() testStream(query)( @@ -686,7 +686,7 @@ class StateStoreCoordinatorStreamingSuite extends StreamTest { SQLConf.STATE_STORE_COORDINATOR_SNAPSHOT_LAG_REPORT_INTERVAL.key -> "0" ) { withTempDir { srcDir => - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val query = inputData.toDF().dropDuplicates() // Populate state stores with an initial snapshot, so that timestamp isn't marked diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index e839ccd35ec0b..232332a6575a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -1213,11 +1213,11 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] test("SPARK-21145: Restarted queries create new provider instances") { try { val checkpointLocation = Utils.createTempDir().getAbsoluteFile - implicit val spark: SparkSession = SparkSession.builder().master("local[2]").getOrCreate() + val spark: SparkSession = SparkSession.builder().master("local[2]").getOrCreate() SparkSession.setActiveSession(spark) spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "1") import spark.implicits._ - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) def runQueryAndGetLoadedProviders(): Seq[StateStoreProvider] = { val aggregated = inputData.toDF().groupBy("value").agg(count("*")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index 93efbe3b3cf5a..4cd3f849a5944 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -827,7 +827,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { def constructUnionDf(desiredPartitionsForInput1: Int) : (MemoryStream[String], MemoryStream[String], DataFrame) = { - val input1 = MemoryStream[String](desiredPartitionsForInput1) + val input1 = MemoryStream[String](spark, desiredPartitionsForInput1) val input2 = MemoryStream[String] val df1 = input1.toDF() .select($"value", $"value") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 7825730d901da..f065f1de5cdc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -347,7 +347,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { " shifted partition IDs") { def constructUnionDf(desiredPartitionsForInput1: Int) : (MemoryStream[Int], MemoryStream[Int], DataFrame) = { - val input1 = MemoryStream[Int](desiredPartitionsForInput1) + val input1 = MemoryStream[Int](spark, desiredPartitionsForInput1) val input2 = MemoryStream[Int] val df1 = input1.toDF() .select($"value", $"value" + 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index 832b22d6304fc..11fc9cbfc484b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -334,7 +334,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { " shifted partition IDs") { def constructUnionDf(desiredPartitionsForInput1: Int) : (MemoryStream[Int], MemoryStream[Int], DataFrame) = { - val input1 = MemoryStream[Int](desiredPartitionsForInput1) + val input1 = MemoryStream[Int](spark, desiredPartitionsForInput1) val input2 = MemoryStream[Int] val df1 = input1.toDF().select($"value") val df2 = input2.toDF().dropDuplicates("value") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 22028a585e229..6cdca9fb5309f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -1609,7 +1609,7 @@ class StreamingOuterJoinSuite extends StreamingJoinSuite { test("SPARK-29438: ensure UNION doesn't lead stream-stream join to use shifted partition IDs") { def constructUnionDf(desiredPartitionsForInput1: Int) : (MemoryStream[Int], MemoryStream[Int], MemoryStream[Int], DataFrame) = { - val input1 = MemoryStream[Int](desiredPartitionsForInput1) + val input1 = MemoryStream[Int](spark, desiredPartitionsForInput1) val df1 = input1.toDF() .select( $"value" as "key", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index baafdc1ea50a3..86041d48cde75 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2655,7 +2655,7 @@ class HiveDDLSuite |SELECT word, number from t1 """.stripMargin) - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int](spark) val joined = inputData.toDS().toDF() .join(spark.table("smallTable"), $"value" === $"number") diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala index 7c8181b5b72a5..f37716b4a24d3 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.pipelines.graph -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.utils.{PipelineTest, TestGraphRegistrationContext} import org.apache.spark.sql.test.SharedSparkSession @@ -423,7 +423,6 @@ class ConnectInvalidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ val p = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem = MemoryStream[Int] mem.addData(1) registerPersistedView("a", query = dfFlowFunc(mem.toDF())) @@ -467,7 +466,6 @@ class ConnectInvalidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ val graph = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark registerMaterializedView("a", query = dfFlowFunc(MemoryStream[Int].toDF())) }.resolveToDataflowGraph() @@ -491,7 +489,6 @@ class ConnectInvalidPipelineSuite extends PipelineTest with SharedSparkSession { val graph = new TestGraphRegistrationContext(spark) { registerTable("a") - implicit val sparkSession: SparkSession = spark registerFlow( destinationName = "a", name = "once_flow", diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala index a4bb7c067d875..3ac3c09017506 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.pipelines.graph -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.Union @@ -159,7 +158,6 @@ class ConnectValidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ class P extends TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val ints = MemoryStream[Int] ints.addData(1, 2, 3, 4) registerPersistedView("a", query = dfFlowFunc(ints.toDF())) @@ -201,7 +199,6 @@ class ConnectValidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ class P extends TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val ints1 = MemoryStream[Int] ints1.addData(1, 2, 3, 4) val ints2 = MemoryStream[Int] @@ -362,7 +359,6 @@ class ConnectValidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ class P extends TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem = MemoryStream[Int] registerPersistedView("a", query = dfFlowFunc(mem.toDF())) registerTable("b") @@ -406,7 +402,6 @@ class ConnectValidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ val graph = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem = MemoryStream[Int] mem.addData(1, 2) registerPersistedView("complete-view", query = dfFlowFunc(Seq(1, 2).toDF("x"))) @@ -499,7 +494,6 @@ class ConnectValidPipelineSuite extends PipelineTest with SharedSparkSession { import session.implicits._ val P = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem = MemoryStream[Int] mem.addData(1, 2) registerTemporaryView("a", query = dfFlowFunc(mem.toDF().select($"value" as "x"))) diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala index ba8419eb6e9c8..72cc644e57684 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.pipelines.graph import scala.jdk.CollectionConverters._ import org.apache.spark.SparkThrowable -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.{ClusterByTransform, Expressions, FieldReference} import org.apache.spark.sql.execution.streaming.runtime.MemoryStream @@ -269,7 +268,7 @@ abstract class MaterializeTablesSuite extends BaseCoreExecutionTest { test("invalid schema merge") { val session = spark - implicit val sparkSession: SparkSession = spark + implicit val sqlCtx: SQLContext = spark.sqlContext import session.implicits._ val streamInts = MemoryStream[Int] @@ -353,7 +352,6 @@ abstract class MaterializeTablesSuite extends BaseCoreExecutionTest { val ex = intercept[TableMaterializationException] { materializeGraph(new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val source: MemoryStream[Int] = MemoryStream[Int] source.addData(1, 2) registerTable( @@ -646,7 +644,7 @@ abstract class MaterializeTablesSuite extends BaseCoreExecutionTest { s"Streaming tables should evolve schema only if not full refresh = $isFullRefresh" ) { val session = spark - implicit val sparkSession: SparkSession = spark + implicit val sqlCtx: SQLContext = spark.sqlContext import session.implicits._ val streamInts = MemoryStream[Int] diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala index c37a6fb52f95d..71301c34c14ef 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/SystemMetadataSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.pipelines.graph import org.apache.hadoop.fs.Path -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.pipelines.utils.{ExecutionTest, TestGraphRegistrationContext} @@ -39,7 +38,6 @@ class SystemMetadataSuite // create a pipeline with only a single ST val graph = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem: MemoryStream[Int] = MemoryStream[Int] mem.addData(1, 2, 3) registerView("a", query = dfFlowFunc(mem.toDF())) @@ -107,7 +105,6 @@ class SystemMetadataSuite import session.implicits._ val graph = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem: MemoryStream[Int] = MemoryStream[Int] mem.addData(1, 2, 3) registerView("a", query = dfFlowFunc(mem.toDF())) @@ -172,7 +169,6 @@ class SystemMetadataSuite import session.implicits._ val graph = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem: MemoryStream[Int] = MemoryStream[Int] mem.addData(1, 2, 3) registerView("a", query = dfFlowFunc(mem.toDF())) @@ -234,7 +230,6 @@ class SystemMetadataSuite // create a pipeline with only a single ST val graph = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark val mem: MemoryStream[Int] = MemoryStream[Int] mem.addData(1, 2, 3) registerView("a", query = dfFlowFunc(mem.toDF())) diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala index 36b749cc84d9f..57baf4c2d5b11 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.pipelines.graph import org.scalatest.time.{Seconds, Span} -import org.apache.spark.sql.{functions, Row, SparkSession} +import org.apache.spark.sql.{functions, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.classic.{DataFrame, Dataset} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} @@ -183,7 +183,6 @@ class TriggeredGraphExecutionSuite extends ExecutionTest with SharedSparkSession // Construct pipeline val pipelineDef = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark private val ints = MemoryStream[Int] ints.addData(1 until 10: _*) registerView("input", query = dfFlowFunc(ints.toDF())) @@ -260,7 +259,6 @@ class TriggeredGraphExecutionSuite extends ExecutionTest with SharedSparkSession // Construct pipeline val pipelineDef = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark private val ints = MemoryStream[Int] registerView("input", query = dfFlowFunc(ints.toDF())) registerTable( @@ -311,7 +309,6 @@ class TriggeredGraphExecutionSuite extends ExecutionTest with SharedSparkSession }) val pipelineDef = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark private val memoryStream = MemoryStream[Int] memoryStream.addData(1, 2) registerView("input_view", query = dfFlowFunc(memoryStream.toDF())) @@ -551,7 +548,6 @@ class TriggeredGraphExecutionSuite extends ExecutionTest with SharedSparkSession // Construct pipeline val pipelineDef = new TestGraphRegistrationContext(spark) { - implicit val sparkSession: SparkSession = spark private val memoryStream = MemoryStream[Int] memoryStream.addData(1, 2) registerView("input_view", query = dfFlowFunc(memoryStream.toDF())) diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala index e7c0956385135..9ff92ee895b1d 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/TestGraphRegistrationContext.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.pipelines.utils +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{LocalTempView, PersistedView => PersistedViewType, UnresolvedRelation, ViewType} import org.apache.spark.sql.classic.{DataFrame, SparkSession} @@ -28,7 +29,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * A test class to simplify the creation of pipelines and datasets for unit testing. */ class TestGraphRegistrationContext( - val spark: SparkSession, + val _spark: SparkSession, val sqlConf: Map[String, String] = Map.empty) extends GraphRegistrationContext( defaultCatalog = TestGraphRegistrationContext.DEFAULT_CATALOG, @@ -36,6 +37,10 @@ class TestGraphRegistrationContext( defaultSqlConf = sqlConf ) { + /** Re-expose as implicit so nested anonymous classes can use it without shadowing issues */ + implicit def spark: SparkSession = _spark + implicit def sqlContext: SQLContext = _spark.sqlContext + // scalastyle:off // Disable scalastyle to ignore argument count. /** Registers a streaming table in this [[TestGraphRegistrationContext]] */ @@ -145,7 +150,7 @@ class TestGraphRegistrationContext( val qualifiedIdentifier = GraphIdentifierManager .parseAndQualifyTableIdentifier( rawTableIdentifier = GraphIdentifierManager - .parseTableIdentifier(name, spark), + .parseTableIdentifier(name, _spark), currentCatalog = catalog.orElse(Some(defaultCatalog)), currentDatabase = database.orElse(Some(defaultDatabase))) .identifier @@ -304,9 +309,9 @@ class TestGraphRegistrationContext( catalog: Option[String] = None, database: Option[String] = None ): Unit = { - val rawFlowIdentifier = GraphIdentifierManager.parseTableIdentifier(name, spark) + val rawFlowIdentifier = GraphIdentifierManager.parseTableIdentifier(name, _spark) val rawDestinationIdentifier = - GraphIdentifierManager.parseTableIdentifier(destinationName, spark) + GraphIdentifierManager.parseTableIdentifier(destinationName, _spark) val flowWritesToView = getViews .filter(_.isInstanceOf[TemporaryView]) From 3b797bc169a050c9160b7cd50bc81aa7b902c98d Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 9 Feb 2026 09:01:27 +0900 Subject: [PATCH 377/400] [SPARK-55434][INFRA] Add username and password at svn with rm at finalize step ### What changes were proposed in this pull request? ``` + OLD_VERSION=4.2.0-preview1 + [[ -n 4.2.0-preview1 ]] + echo 'Removing old version: spark-4.2.0-preview1' Removing old version: spark-4.2.0-preview1 + svn rm https://dist.apache.org/repos/dist/release/spark/spark-4.2.0-preview1 -m 'Remove older 4.2 release after 4.2.0-preview2' svn: E215004: Authentication failed and interactive prompting is disabled; see the --force-interactive option svn: E215004: No more credentials or we tried too many times. Authentication failed ``` We need to set password and username. ### Why are the changes needed? Add username and password at svn with rm at finalize step ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54217 from HyukjinKwon/fix-release-script. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit f6031fef94f30f521000985f8bf5a3f403b31db1) Signed-off-by: Hyukjin Kwon --- dev/create-release/release-build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 9a83eab694607..d4099a9563ae1 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -565,7 +565,7 @@ EOF if [[ -n "$OLD_VERSION" ]]; then echo "Removing old version: spark-$OLD_VERSION" - svn rm "https://dist.apache.org/repos/dist/release/spark/spark-$OLD_VERSION" -m "Remove older $RELEASE_SERIES release after $RELEASE_VERSION" + svn rm "https://dist.apache.org/repos/dist/release/spark/spark-$OLD_VERSION" --username "$ASF_USERNAME" --password "$ASF_PASSWORD" --non-interactive -m "Remove older $RELEASE_SERIES release after $RELEASE_VERSION" else echo "No previous $RELEASE_SERIES version found to remove. Manually remove it if there is." fi From d4d03469946487a5ff9d0b67e9ab57b8f4b49d83 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 11 Feb 2026 08:55:17 +0100 Subject: [PATCH 378/400] [SPARK-55411][SQL][4.1] SPJ may throw ArrayIndexOutOfBoundsException when join keys are less than cluster keys Backport https://github.com/apache/spark/issues/54182 to branch-4.1 ### What changes were proposed in this pull request? Fix a `java.lang.ArrayIndexOutOfBoundsException` when `spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled=true`, by correcting the `expression`(should pass the full partition expression instead of the projected one) passed to `KeyGroupedPartitioning#project`. Also, fix a test code issue, change the calculation result of `BucketTransform` defined at `InMemoryBaseTable.scala` to match `BucketFunctions` defined at `transformFunctions.scala` (thanks peter-toth for pointing this out!) ### Why are the changes needed? It's a bug fix. ### Does this PR introduce _any_ user-facing change? Some queries that failed when `spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled=true` now run normally. ### How was this patch tested? New UT is added, previously it failed with `ArrayIndexOutOfBoundsException`, now passed. ``` $ build/sbt "sql/testOnly *KeyGroupedPartitioningSuite -- -z SPARK=55411" ... [info] - bug *** FAILED *** (1 second, 884 milliseconds) [info] java.lang.ArrayIndexOutOfBoundsException: Index 1 out of bounds for length 1 [info] at scala.collection.immutable.ArraySeq$ofRef.apply(ArraySeq.scala:331) [info] at org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning$.$anonfun$project$1(partitioning.scala:471) [info] at org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning$.$anonfun$project$1$adapted(partitioning.scala:471) [info] at scala.collection.immutable.ArraySeq.map(ArraySeq.scala:75) [info] at scala.collection.immutable.ArraySeq.map(ArraySeq.scala:35) [info] at org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning$.project(partitioning.scala:471) [info] at org.apache.spark.sql.execution.KeyGroupedPartitionedScan.$anonfun$getOutputKeyGroupedPartitioning$5(KeyGroupedPartitionedScan.scala:58) ... ``` UTs affected by `bucket()` calculate logic change are tuned. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54259 from pan3793/SPARK-55411-4.1. Authored-by: Cheng Pan Signed-off-by: Peter Toth --- .../connector/catalog/InMemoryBaseTable.scala | 27 ++++++++---- .../execution/KeyGroupedPartitionedScan.scala | 8 ++-- .../KeyGroupedPartitioningSuite.scala | 42 +++++++++++++++++-- .../sql/connector/MetadataColumnSuite.scala | 16 +++---- .../functions/transformFunctions.scala | 3 +- 5 files changed, 72 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala index 18fe80c2e924e..29b336c780f8d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala @@ -236,15 +236,26 @@ abstract class InMemoryBaseTable( case (v, t) => throw new IllegalArgumentException(s"Match: unsupported argument(s) type - ($v, $t)") } + // the result should be consistent with BucketFunctions defined at transformFunctions.scala case BucketTransform(numBuckets, cols, _) => - val valueTypePairs = cols.map(col => extractor(col.fieldNames, cleanedSchema, row)) - var valueHashCode = 0 - valueTypePairs.foreach( pair => - if ( pair._1 != null) valueHashCode += pair._1.hashCode() - ) - var dataTypeHashCode = 0 - valueTypePairs.foreach(dataTypeHashCode += _._2.hashCode()) - ((valueHashCode + 31 * dataTypeHashCode) & Integer.MAX_VALUE) % numBuckets + val hash: Long = cols.foldLeft(0L) { (acc, col) => + val valueHash = extractor(col.fieldNames, cleanedSchema, row) match { + case (value: Byte, _: ByteType) => value.toLong + case (value: Short, _: ShortType) => value.toLong + case (value: Int, _: IntegerType) => value.toLong + case (value: Long, _: LongType) => value + case (value: Long, _: TimestampType) => value + case (value: Long, _: TimestampNTZType) => value + case (value: UTF8String, _: StringType) => + value.hashCode.toLong + case (value: Array[Byte], BinaryType) => + util.Arrays.hashCode(value).toLong + case (v, t) => + throw new IllegalArgumentException(s"Match: unsupported argument(s) type - ($v, $t)") + } + acc + valueHash + } + Math.floorMod(hash, numBuckets) case NamedTransform("truncate", Seq(ref: NamedReference, length: V2Literal[_])) => extractor(ref.fieldNames, cleanedSchema, row) match { case (str: UTF8String, StringType) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala index 5a789179219ad..e9d8e8e6d0fb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala @@ -34,7 +34,7 @@ trait KeyGroupedPartitionedScan[T] { def getOutputKeyGroupedPartitioning( basePartitioning: KeyGroupedPartitioning, spjParams: StoragePartitionJoinParams): KeyGroupedPartitioning = { - val expressions = spjParams.joinKeyPositions match { + val projectedExpressions = spjParams.joinKeyPositions match { case Some(projectionPositions) => projectionPositions.map(i => basePartitioning.expressions(i)) case _ => basePartitioning.expressions @@ -50,14 +50,14 @@ trait KeyGroupedPartitionedScan[T] { case None => spjParams.joinKeyPositions match { case Some(projectionPositions) => basePartitioning.partitionValues.map { r => - val projectedRow = KeyGroupedPartitioning.project(expressions, + val projectedRow = KeyGroupedPartitioning.project(basePartitioning.expressions, projectionPositions, r) - InternalRowComparableWrapper(projectedRow, expressions) + InternalRowComparableWrapper(projectedRow, projectedExpressions) }.distinct.map(_.row) case _ => basePartitioning.partitionValues } } - basePartitioning.copy(expressions = expressions, numPartitions = newPartValues.length, + basePartitioning.copy(expressions = projectedExpressions, numPartitions = newPartValues.length, partitionValues = newPartValues) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 7c830bf6c6e18..274741fb8e7be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -124,7 +124,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32)))) // Has exactly one partition. - val partitionValues = Seq(31).map(v => InternalRow.fromSeq(Seq(v))) + val partitionValues = Seq(0).map(v => InternalRow.fromSeq(Seq(v))) checkQueryPlan(df, distribution, physical.KeyGroupedPartitioning(distribution.clustering, 1, partitionValues, partitionValues)) } @@ -2798,8 +2798,6 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { } test("SPARK-54439: KeyGroupedPartitioning with transform and join key size mismatch") { - // Do not use `bucket()` in "one side partition" tests as its implementation in - // `InMemoryBaseTable` conflicts with `BucketFunction` val items_partitions = Array(years("arrive_time")) createTable(items, itemsColumns, items_partitions) @@ -2823,4 +2821,42 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0))) } } + + test("SPARK-55411: Fix ArrayIndexOutOfBoundsException when join keys " + + "are less than cluster keys") { + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> "false", + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> "true") { + + val customers_partitions = Array(identity("customer_name"), bucket(4, "customer_id")) + createTable(customers, customersColumns, customers_partitions) + sql(s"INSERT INTO testcat.ns.$customers VALUES " + + s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)") + + createTable(orders, ordersColumns, Array.empty) + sql(s"INSERT INTO testcat.ns.$orders VALUES " + + s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)") + + val df = sql( + s"""${selectWithMergeJoinHint("c", "o")} + |customer_name, customer_age, order_amount + |FROM testcat.ns.$customers c JOIN testcat.ns.$orders o + |ON c.customer_id = o.customer_id ORDER BY c.customer_id, order_amount + |""".stripMargin) + + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.length == 1) + + checkAnswer(df, Seq( + Row("aaa", 10, 100.0), + Row("aaa", 10, 200.0), + Row("bbb", 20, 150.0), + Row("bbb", 20, 250.0), + Row("bbb", 20, 350.0), + Row("ccc", 30, 400.50))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala index 741e30a739f5e..7580d524e7ff5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala @@ -41,7 +41,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { val dfQuery = spark.table(tbl).select("id", "data", "index", "_partition") Seq(sqlQuery, dfQuery).foreach { query => - checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + checkAnswer(query, Seq(Row(1, "a", 0, "1/1"), Row(2, "b", 0, "2/2"), Row(3, "c", 0, "3/3"))) } } } @@ -55,7 +55,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { val dfQuery = spark.table(tbl).select("index", "data", "_partition") Seq(sqlQuery, dfQuery).foreach { query => - checkAnswer(query, Seq(Row(3, "c", "1/3"), Row(2, "b", "0/2"), Row(1, "a", "3/1"))) + checkAnswer(query, Seq(Row(3, "c", "3/3"), Row(2, "b", "2/2"), Row(1, "a", "1/1"))) } } } @@ -124,7 +124,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { checkAnswer( dfQuery, - Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3")) + Seq(Row(1, "a", 0, "1/1"), Row(2, "b", 0, "2/2"), Row(3, "c", 0, "3/3")) ) } } @@ -134,7 +134,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { prepareTable() checkAnswer( spark.table(tbl).select("id", "data").select("index", "_partition"), - Seq(Row(0, "3/1"), Row(0, "0/2"), Row(0, "1/3")) + Seq(Row(0, "1/1"), Row(0, "2/2"), Row(0, "3/3")) ) } } @@ -159,7 +159,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { val dfQuery = spark.table(tbl).where("id > 1").select("id", "data", "index", "_partition") Seq(sqlQuery, dfQuery).foreach { query => - checkAnswer(query, Seq(Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + checkAnswer(query, Seq(Row(2, "b", 0, "2/2"), Row(3, "c", 0, "3/3"))) } } } @@ -171,7 +171,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { val dfQuery = spark.table(tbl).orderBy("id").select("id", "data", "index", "_partition") Seq(sqlQuery, dfQuery).foreach { query => - checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + checkAnswer(query, Seq(Row(1, "a", 0, "1/1"), Row(2, "b", 0, "2/2"), Row(3, "c", 0, "3/3"))) } } } @@ -185,7 +185,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { .select("id", "data", "index", "_partition") Seq(sqlQuery, dfQuery).foreach { query => - checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + checkAnswer(query, Seq(Row(1, "a", 0, "1/1"), Row(2, "b", 0, "2/2"), Row(3, "c", 0, "3/3"))) } } } @@ -200,7 +200,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { s"$sbq.id", s"$sbq.data", s"$sbq.index", s"$sbq._partition") Seq(sqlQuery, dfQuery).foreach { query => - checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + checkAnswer(query, Seq(Row(1, "a", 0, "1/1"), Row(2, "b", 0, "2/2"), Row(3, "c", 0, "3/3"))) } assertThrows[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/catalog/functions/transformFunctions.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/catalog/functions/transformFunctions.scala index b82cc2392e1fc..ed2f81d7e8d6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/catalog/functions/transformFunctions.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/catalog/functions/transformFunctions.scala @@ -84,6 +84,7 @@ object UnboundBucketFunction extends UnboundFunction { override def name(): String = "bucket" } +// the result should be consistent with BucketTransform defined at InMemoryBaseTable.scala object BucketFunction extends ScalarFunction[Int] with ReducibleFunction[Int, Int] { override def inputTypes(): Array[DataType] = Array(IntegerType, LongType) override def resultType(): DataType = IntegerType @@ -91,7 +92,7 @@ object BucketFunction extends ScalarFunction[Int] with ReducibleFunction[Int, In override def canonicalName(): String = name() override def toString: String = name() override def produceResult(input: InternalRow): Int = { - (input.getLong(1) % input.getInt(0)).toInt + Math.floorMod(input.getLong(1), input.getInt(0)) } override def reducer( From 74d2f0d13ac58a9aab5feedb826362fb1eef49ce Mon Sep 17 00:00:00 2001 From: Chris Boumalhab Date: Wed, 11 Feb 2026 13:09:49 -0800 Subject: [PATCH 379/400] [SPARK-52407][SQL][FOLLOW-UP] Remove Theta Sketch aggregation buffer re-wrapping ### What changes were proposed in this pull request? The current Theta sketch update and merge functions from `TypedImperativeAggregate` unnecessarily re-wrap the aggregation buffer with one of the ThetaSketchState case classes. Since changes to the buffer are mutable, we can avoid this re-wrap entirely. ### Why are the changes needed? Better engineering practice, small optimization. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? re-ran the SQLQueryTestSuite with `SPARK_GENERATE_GOLDEN_FILES=1`, no impact as expected. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53984 from cboumalh/cboumalh-theta-merge-followup. Authored-by: Chris Boumalhab Signed-off-by: Daniel Tenedorio (cherry picked from commit 6112a0bfc4818a161bc0f69bcb7a5351fa9ba65a) Signed-off-by: Daniel Tenedorio --- .../aggregate/thetasketchesAggregates.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala index a14df39bf822d..0f148d03cd70b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala @@ -219,7 +219,7 @@ case class ThetaSketchAgg( messageParameters = Map("dataType" -> left.dataType.toString)) } - UpdatableSketchBuffer(sketch) + updateBuffer } /** @@ -246,13 +246,13 @@ case class ThetaSketchAgg( // Reuse the existing union in the next iteration. This is the most efficient path. case (UnionAggregationBuffer(existingUnion), UpdatableSketchBuffer(sketch)) => existingUnion.union(sketch.compact) - UnionAggregationBuffer(existingUnion) + updateBuffer case (UnionAggregationBuffer(existingUnion), FinalizedSketch(sketch)) => existingUnion.union(sketch) - UnionAggregationBuffer(existingUnion) + updateBuffer case (UnionAggregationBuffer(union1), UnionAggregationBuffer(union2)) => union1.union(union2.getResult) - UnionAggregationBuffer(union1) + updateBuffer // Create a new union only when necessary. case (UpdatableSketchBuffer(sketch1), UpdatableSketchBuffer(sketch2)) => createUnionWith(sketch1.compact, sketch2.compact) @@ -420,7 +420,7 @@ case class ThetaUnionAgg( case _ => throw QueryExecutionErrors.thetaInvalidInputSketchBuffer(prettyName) } union.union(inputSketch) - UnionAggregationBuffer(union) + unionBuffer } /** @@ -436,11 +436,11 @@ case class ThetaUnionAgg( // If both arguments are union objects, merge them directly. case (UnionAggregationBuffer(unionLeft), UnionAggregationBuffer(unionRight)) => unionLeft.union(unionRight.getResult) - UnionAggregationBuffer(unionLeft) + unionBuffer // The input was serialized then deserialized. case (UnionAggregationBuffer(union), FinalizedSketch(sketch)) => union.union(sketch) - UnionAggregationBuffer(union) + unionBuffer // The program should never make it here, the cases are for defensive programming. case (FinalizedSketch(sketch1), FinalizedSketch(sketch2)) => val union = SetOperation.builder.setLogNominalEntries(lgNomEntries).buildUnion @@ -449,7 +449,7 @@ case class ThetaUnionAgg( UnionAggregationBuffer(union) case (FinalizedSketch(sketch), UnionAggregationBuffer(union)) => union.union(sketch) - UnionAggregationBuffer(union) + input case _ => throw QueryExecutionErrors.thetaInvalidInputSketchBuffer(prettyName) } } @@ -582,7 +582,7 @@ case class ThetaIntersectionAgg( case _ => throw QueryExecutionErrors.thetaInvalidInputSketchBuffer(prettyName) } intersection.intersect(inputSketch) - IntersectionAggregationBuffer(intersection) + intersectionBuffer } /** @@ -603,11 +603,11 @@ case class ThetaIntersectionAgg( IntersectionAggregationBuffer(intersectLeft), IntersectionAggregationBuffer(intersectRight)) => intersectLeft.intersect(intersectRight.getResult) - IntersectionAggregationBuffer(intersectLeft) + intersectionBuffer // The input was serialized then deserialized. case (IntersectionAggregationBuffer(intersection), FinalizedSketch(sketch)) => intersection.intersect(sketch) - IntersectionAggregationBuffer(intersection) + intersectionBuffer // The program should never make it here, the cases are for defensive programming. case (FinalizedSketch(sketch1), FinalizedSketch(sketch2)) => val intersection = @@ -617,7 +617,7 @@ case class ThetaIntersectionAgg( IntersectionAggregationBuffer(intersection) case (FinalizedSketch(sketch), IntersectionAggregationBuffer(intersection)) => intersection.intersect(sketch) - IntersectionAggregationBuffer(intersection) + input case _ => throw QueryExecutionErrors.thetaInvalidInputSketchBuffer(prettyName) } } From ea11e50114b058c76dcb18d25170a52039c37784 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 12 Feb 2026 11:07:41 -0800 Subject: [PATCH 380/400] [SPARK-55495][CORE] Fix `EventLogFileWriters.closeWriter` to handle `checkError` ### What changes were proposed in this pull request? This PR aims to fix `EventLogFileWriters.closeWriter` to handle `checkError`. In general, we need the following three. 1. Do `flush` first before closing to isolate any problems at this layer. 2. Do `PrintWriter.close` and fallback to the underlying Hadoop file stream's `close` API. 3. Show warnings properly if `checkError` returns true. ### Why are the changes needed? Currently, Apache Spark's event log writer naively invokes `PrintWriter.close()` without error handling. https://github.com/apache/spark/blob/4e1cb88bba0c031f54dd07e3adc0d464d45cbfce/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala#L80 https://github.com/apache/spark/blob/4e1cb88bba0c031f54dd07e3adc0d464d45cbfce/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala#L133-L135 However, Java community recommends to use `checkError` in case of `PrintWriter.flush` and `PrintWriter.close`. - https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/io/PrintWriter.html#checkError() When `checkError` returns `true`, a user can lose their event log. For example, the event log is not uploaded silently. Spark had better show a proper warning and tries to do the best efforts to flush or close the underlying Hadoop File streams at least. ### Does this PR introduce _any_ user-facing change? No, this is a bug fix for the corner case. ### How was this patch tested? Pass the CIs with the newly added test cases. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: `Opus 4.5` on `Claude Code` Closes #54280 from dongjoon-hyun/SPARK-55495. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 3484a4acf13aafb05f49a51a2adef9c38d5daacc) Signed-off-by: Dongjoon Hyun --- .../deploy/history/EventLogFileWriters.scala | 13 ++ .../history/EventLogFileWritersSuite.scala | 146 +++++++++++++++++- 2 files changed, 158 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 4e3bee1015ff3..7c022c283db41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -131,7 +131,20 @@ abstract class EventLogFileWriter( } protected def closeWriter(): Unit = { + // 1. Flush first to check the errors + writer.foreach(_.flush()) + if (writer.exists(_.checkError())) { + logError("Spark detects errors while flushing event logs.") + } + hadoopDataStream.foreach(_.hflush()) + + // 2. Try to close and check the errors writer.foreach(_.close()) + if (writer.exists(_.checkError())) { + logError("Spark detects errors while closing event logs.") + // 3. Ensuring the underlying stream is closed at least (best-effort). + hadoopDataStream.foreach(_.close()) + } } protected def renameFile(src: Path, dest: Path, overwrite: Boolean): Unit = { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index d9d6a4f8d35df..00a92c503be4e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.history -import java.io.{File, FileOutputStream, IOException} +import java.io.{File, FileOutputStream, IOException, OutputStream, PrintWriter} import java.net.URI import scala.collection.mutable @@ -160,8 +160,152 @@ abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkCon expectedLines: Seq[String] = Seq.empty): Unit } +/** + * A test OutputStream that simulates IO errors. + */ +class ErrorThrowingOutputStream extends OutputStream { + var throwOnWrite: Boolean = false + var throwOnFlush: Boolean = false + var throwOnClose: Boolean = false + + override def write(b: Int): Unit = { + if (throwOnWrite) { + throw new IOException("Simulated write error") + } + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + if (throwOnWrite) { + throw new IOException("Simulated write error") + } + } + + override def flush(): Unit = { + if (throwOnFlush) { + throw new IOException("Simulated flush error") + } + } + + override def close(): Unit = { + if (throwOnClose) { + throw new IOException("Simulated close error") + } + } +} + +/** + * A testable subclass of SingleEventLogFileWriter that exposes the writer field + * and closeWriter method for testing. + */ +class TestableSingleEventLogFileWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) + extends SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { + + def setWriterForTest(pw: PrintWriter): Unit = { + writer = Some(pw) + } + + def callCloseWriter(): Unit = { + closeWriter() + } +} + class SingleEventLogFileWriterSuite extends EventLogFileWritersSuite { + test("SPARK-55495: closeWriter should log warning when flush error occurs") { + val appId = getUniqueApplicationId + val attemptId = None + val conf = getLoggingConf(testDirPath, None) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val writer = new TestableSingleEventLogFileWriter( + appId, attemptId, testDirPath.toUri, conf, hadoopConf) + + // Create a PrintWriter with an ErrorThrowingOutputStream + val errorStream = new ErrorThrowingOutputStream() + val printWriter = new PrintWriter(errorStream) + + // Simulate an error by writing to a closed stream that causes checkError to return true + errorStream.throwOnWrite = true + // scalastyle:off println + printWriter.println("test") // This will set the error flag + // scalastyle:on println + + writer.setWriterForTest(printWriter) + + val logAppender = new LogAppender("closeWriter flush error test") + withLogAppender(logAppender, level = Some(org.apache.logging.log4j.Level.WARN)) { + writer.callCloseWriter() + } + + val warningMessages = logAppender.loggingEvents.map(_.getMessage.getFormattedMessage) + assert(warningMessages.exists(_.contains("Spark detects errors while flushing")), + s"Expected warning message not found. Messages: $warningMessages") + } + + test("SPARK-55495: closeWriter should log warning when close error occurs") { + val appId = getUniqueApplicationId + val attemptId = None + val conf = getLoggingConf(testDirPath, None) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val writer = new TestableSingleEventLogFileWriter( + appId, attemptId, testDirPath.toUri, conf, hadoopConf) + + // Create a PrintWriter with an ErrorThrowingOutputStream that errors on close + val errorStream = new ErrorThrowingOutputStream() + val printWriter = new PrintWriter(errorStream) + + // First write something successfully + // scalastyle:off println + printWriter.println("test") + // scalastyle:on println + printWriter.flush() + + // Now set up to error on close + errorStream.throwOnClose = true + + writer.setWriterForTest(printWriter) + + val logAppender = new LogAppender("closeWriter close error test") + withLogAppender(logAppender, level = Some(org.apache.logging.log4j.Level.WARN)) { + writer.callCloseWriter() + } + + val warningMessages = logAppender.loggingEvents.map(_.getMessage.getFormattedMessage) + assert(warningMessages.exists(_.contains("Spark detects errors while closing")), + s"Expected warning message not found. Messages: $warningMessages") + } + + test("SPARK-55495: closeWriter should complete without warnings when no errors") { + val appId = getUniqueApplicationId + val attemptId = None + val conf = getLoggingConf(testDirPath, None) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val writer = new TestableSingleEventLogFileWriter( + appId, attemptId, testDirPath.toUri, conf, hadoopConf) + + // Create a normal PrintWriter with no errors + val normalStream = new ErrorThrowingOutputStream() + val printWriter = new PrintWriter(normalStream) + + writer.setWriterForTest(printWriter) + + val logAppender = new LogAppender("closeWriter no error test") + withLogAppender(logAppender, level = Some(org.apache.logging.log4j.Level.WARN)) { + writer.callCloseWriter() + } + + val warningMessages = logAppender.loggingEvents.map(_.getMessage.getFormattedMessage) + assert(!warningMessages.exists(_.contains("Spark detects errors")), + s"Unexpected warning message found. Messages: $warningMessages") + } + test("Log overwriting") { val appId = "test" val appAttemptId = None From 68cdbfec3048bf935af2de66f64fdbc947f490ad Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 13 Feb 2026 10:14:21 +0800 Subject: [PATCH 381/400] [SPARK-55497][BUILD][TESTS] Use `jupyterTestFramework` instead of `TestFrameworks.JUnit` ### What changes were proposed in this pull request? This PR fixes an issue that `TestFrameworks.JUnit` is still used even though newer jupiter plugin (`com.github.sbt.junit.sbt-jupiter-interface`) is used. As a result, `-Dtests.include.tags` and `-Dtests.exclude.tags` don't work. ``` $ build/sbt -Dtest.include.tags=org.apache.spark.tags.SlowHiveTest unsafe/test ... [info] Run completed in 2 seconds, 877 milliseconds. [info] Total number of tests run: 0 [info] Suites: completed 2, aborted 0 [info] Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0 [info] No tests were executed. [info] Passed: Total 97, Failed 0, Errors 0, Passed 97 ``` ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Confirmed `-Dtests.exclude.tags` works. ``` $ build/sbt -Dtest.include.tags=org.apache.spark.tags.SlowHiveTest unsafe/test ... [info] Run completed in 738 milliseconds. [info] Total number of tests run: 0 [info] Suites: completed 2, aborted 0 [info] Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0 [info] No tests were executed. [info] Passed: Total 0, Failed 0, Errors 0, Passed 0 [success] Total time: 2 s, completed 2026/02/12 19:33:53 ``` There are no tagged `.java` tests so I didn't confirm if `-Dtest.exclude.tags` works. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54281 from sarutak/fix-jupiter-junit5. Authored-by: Kousuke Saruta Signed-off-by: Cheng Pan (cherry picked from commit 448681f88fecf7ebf0c0127b40cc654ff946cd46) Signed-off-by: Cheng Pan --- project/SparkBuild.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6c37944fb48aa..bb505d390e5ee 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -31,6 +31,7 @@ import sbt.Classpaths.publishOrSkip import sbt.Keys._ import sbt.librarymanagement.{ VersionNumber, SemanticSelector } import com.etsy.sbt.checkstyle.CheckstylePlugin.autoImport._ +import com.github.sbt.junit.jupiter.sbt.JupiterPlugin.autoImport._ import com.here.bom.Bom import com.simplytyped.Antlr4Plugin._ import sbtpomreader.{PomBuild, SbtPomKeys} @@ -1902,25 +1903,25 @@ object TestSettings { sys.props.get("test.default.exclude.tags").map(tags => tags.split(",").toSeq) .map(tags => tags.filter(!_.trim.isEmpty)).getOrElse(defaultExcludedTags) .flatMap(tag => Seq("-l", tag)): _*), - (Test / testOptions) += Tests.Argument(TestFrameworks.JUnit, + (Test / testOptions) += Tests.Argument(jupiterTestFramework, sys.props.get("test.exclude.tags").map { tags => - Seq("--exclude-categories=" + tags) + Seq("--exclude-tag=" + tags) }.getOrElse(Nil): _*), // Include tags defined in a system property (Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, sys.props.get("test.include.tags").map { tags => tags.split(",").flatMap { tag => Seq("-n", tag) }.toSeq }.getOrElse(Nil): _*), - (Test / testOptions) += Tests.Argument(TestFrameworks.JUnit, + (Test / testOptions) += Tests.Argument(jupiterTestFramework, sys.props.get("test.include.tags").map { tags => - Seq("--include-categories=" + tags) + Seq("--include-tags=" + tags) }.getOrElse(Nil): _*), // Show full stack trace and duration in test cases. (Test / testOptions) += Tests.Argument("-oDF"), // Slowpoke notifications: receive notifications every 5 minute of tests that have been running // longer than two minutes. (Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, "-W", "120", "300"), - (Test / testOptions) += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), + (Test / testOptions) += Tests.Argument(jupiterTestFramework, "-v", "-a"), // Enable Junit testing. libraryDependencies += "com.github.sbt.junit" % "jupiter-interface" % "0.17.0" % "test", // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project From 4b884f2ea1cfc05b21a4f7c94a5849f245b23c26 Mon Sep 17 00:00:00 2001 From: tugce-applied Date: Tue, 17 Feb 2026 22:14:48 +0800 Subject: [PATCH 382/400] [SPARK-55552][SQL] Add VariantType support to ColumnarBatchRow.copy() and MutableColumnarRow ### What changes were proposed in this pull request? `ColumnarBatchRow.copy()` and `MutableColumnarRow.copy()`/`get()` do not handle `VariantType`, causing a `RuntimeException: Not implemented. VariantType` when using `VariantType` columns with streaming custom data sources that rely on columnar batch row copying. PR #53137 (SPARK-54427) added `VariantType` support to `ColumnarRow` but missed `ColumnarBatchRow` and `MutableColumnarRow`. PR #54006 attempted this fix but was closed. This patch adds: - `PhysicalVariantType` branch in `ColumnarBatchRow.copy()` - `VariantType` branch in `MutableColumnarRow.copy()` and `get()` - Test in `ColumnarBatchSuite` validating `VariantVal` round-trip through `copy()` ### Why are the changes needed? Without this fix, any streaming pipeline that returns `VariantType` columns from a custom columnar data source throws a runtime exception when Spark attempts to copy the batch row. ### Does this PR introduce _any_ user-facing change? No. This is a bug fix for an existing feature. ### How was this patch tested? Added a new test `[SPARK-55552] Variant` in `ColumnarBatchSuite` that creates a `VariantType` column vector, populates it with `VariantVal` data (including a null), wraps it in a `ColumnarBatchRow`, calls `copy()`, and verifies the values round-trip correctly. ### Was this patch authored or co-authored using generative AI tooling? Yes. GitHub Copilot was used to assist in drafting portions of this contribution. This contribution is my original work and I license it under the Apache 2.0 license. Closes #54337 from tugceozberk/fix-columnar-batch-row-variant-copy. Authored-by: tugce-applied Signed-off-by: Wenchen Fan (cherry picked from commit c4188b0e43182e4585ee09cbf3cd00d633ec72e7) Signed-off-by: Wenchen Fan --- .../sql/vectorized/ColumnarBatchRow.java | 2 ++ .../vectorized/MutableColumnarRow.java | 4 +++ .../vectorized/ColumnarBatchSuite.scala | 35 +++++++++++++++++++ 3 files changed, 41 insertions(+) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatchRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatchRow.java index 4be45dc5d399d..3d1e780f6e057 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatchRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatchRow.java @@ -86,6 +86,8 @@ public InternalRow copy() { row.update(i, getArray(i).copy()); } else if (pdt instanceof PhysicalMapType) { row.update(i, getMap(i).copy()); + } else if (pdt instanceof PhysicalVariantType) { + row.update(i, getVariant(i)); } else { throw new RuntimeException("Not implemented. " + dt); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java index 49c27f9775624..a46b5143eef6d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java @@ -96,6 +96,8 @@ public InternalRow copy() { row.update(i, getArray(i).copy()); } else if (dt instanceof MapType) { row.update(i, getMap(i).copy()); + } else if (dt instanceof VariantType) { + row.update(i, getVariant(i)); } else { throw new RuntimeException("Not implemented. " + dt); } @@ -217,6 +219,8 @@ public Object get(int ordinal, DataType dataType) { return getStruct(ordinal, structType.fields().length); } else if (dataType instanceof MapType) { return getMap(ordinal); + } else if (dataType instanceof VariantType) { + return getVariant(ordinal); } else { throw new SparkUnsupportedOperationException( "_LEGACY_ERROR_TEMP_3192", Map.of("dt", dataType.toString())); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 97ad2c1f5bf96..6d90bb985e269 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -2025,4 +2025,39 @@ class ColumnarBatchSuite extends SparkFunSuite { } } } + + testVector("[SPARK-55552] Variant", 3, VariantType) { + column => + val valueChild = column.getChild(0) + val metadataChild = column.getChild(1) + + column.putNotNull(0) + valueChild.appendByteArray(Array[Byte](1, 2, 3), 0, 3) + metadataChild.appendByteArray(Array[Byte](10, 11), 0, 2) + + column.putNotNull(1) + valueChild.appendByteArray(Array[Byte](4, 5), 0, 2) + metadataChild.appendByteArray(Array[Byte](12, 13, 14), 0, 3) + + column.putNull(2) + valueChild.appendNull() + metadataChild.appendNull() + + val batchRow = new ColumnarBatchRow(Array(column)) + (0 until 3).foreach { i => + batchRow.rowId = i + val batchRowCopy = batchRow.copy() + if (i < 2) { + assert(!batchRow.isNullAt(0)) + assert(!batchRowCopy.isNullAt(0)) + val original = batchRow.getVariant(0) + val copied = batchRowCopy.get(0, VariantType).asInstanceOf[VariantVal] + assert(java.util.Arrays.equals(original.getValue, copied.getValue)) + assert(java.util.Arrays.equals(original.getMetadata, copied.getMetadata)) + } else { + assert(batchRow.isNullAt(0)) + assert(batchRowCopy.isNullAt(0)) + } + } + } } From 21ff6457ac7ec38ca0b821f4fe4de6bd56ef3d2d Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 19 Feb 2026 08:47:26 +0900 Subject: [PATCH 383/400] [SPARK-55599][BUILD] Replaces dots to hyphens in release announcement pages MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR proposes to fix the file name back to with hyphens, see ``` [TXT] | spark-release-4-0-0.html | 2026-02-10 09:21 | 134K |   -- | -- | -- | -- | --   | spark-release-4-0-1.html | 2026-02-10 09:21 | 25K |     | spark-release-4-0-2.html | 2026-02-10 09:21 | 11K |     | spark-release-4.1.0.html | 2026-02-10 09:21 | 69K |   [TXT] [spark-release-4-0-0.html](https://spark.apache.org/releases/spark-release-4-0-0.html) 2026-02-10 09:21 134K [TXT] [spark-release-4-0-1.html](https://spark.apache.org/releases/spark-release-4-0-1.html) 2026-02-10 09:21 25K [TXT] [spark-release-4-0-2.html](https://spark.apache.org/releases/spark-release-4-0-2.html) 2026-02-10 09:21 11K [TXT] [spark-release-4.1.0.html](https://spark.apache.org/releases/spark-release-4.1.0.html) 2026-02-10 09:21 69K ``` at https://spark.apache.org/releases/ ### Why are the changes needed? For consistency. ### Does this PR introduce _any_ user-facing change? Virtually no. ### How was this patch tested? Will be tested in the next release. Also manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54372 from HyukjinKwon/SPARK-55599. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit c78b1a1574b0c3747c47502f286de540e5c4ee41) Signed-off-by: Hyukjin Kwon --- dev/create-release/release-build.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index d4099a9563ae1..1a80191cba4bb 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -344,7 +344,7 @@ meta: _edit_last: '4' _wpas_done_all: '1' --- -We are happy to announce the availability of
Apache Spark ${RELEASE_VERSION}! Visit the release notes to read about the new features, or download the release today. +We are happy to announce the availability of Apache Spark ${RELEASE_VERSION}! Visit the release notes to read about the new features, or download the release today. EOF fi @@ -402,7 +402,7 @@ You can find the list of resolved issues and detailed changes in the [JIRA relea We would like to acknowledge all community members for contributing ${ACKNOWLEDGE}" - FILENAME="releases/_posts/${RELEASE_DATE}-spark-release-${RELEASE_VERSION}.md" + FILENAME="releases/_posts/${RELEASE_DATE}-spark-release-${RELEASE_VERSION//./-}.md" mkdir -p releases/_posts cat > "$FILENAME" < Date: Thu, 19 Feb 2026 12:13:46 +0900 Subject: [PATCH 384/400] [SPARK-55416][SS][PYTHON][4.1] Streaming Python Data Source memory leak when end-offset is not updated ### What changes were proposed in this pull request? Backport https://github.com/apache/spark/pull/54237 to branch-4.1 In `_SimpleStreamReaderWrapper.latestOffset()`, validate that custom implementation of datasource based on `SimpleDataSourceStreamReader.read()` does not return a non-empty batch with end == start. If it does, raise PySparkException with error class `SIMPLE_STREAM_READER_OFFSET_DID_NOT_ADVANCE` before appending to the cache. Empty batches with end == start remain allowed. ### Why are the changes needed? When a user implements read(start) incorrectly and returns: - Same offset for both: end = start (e.g. both {"offset": 0}). - Non-empty iterator: e.g. 2 rows. If a reader returns end == start with data (e.g. return (it, {"offset": start_idx})), the wrapper keeps appending to its prefetch cache on every trigger while commit(end) never trims entries (first matching index is 0). The cache grows without bound and driver (non-JVM) memory increases until OOM. Validating and raising error before appending stops this and fails fast with a clear error. Empty batches with end == start remain allowed , it will allow the Python data source to represent that there is no data to read. ### Does this PR introduce _any_ user-facing change? Yes. Implementations that return end == start with a non-empty iterator now get PySparkException instead of unbounded memory growth. Empty batches with end == start are unchanged. ### How was this patch tested? Added unit test in `test_python_streaming_datasource.py` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54321 from vinodkc/br_SPARK-55416_4.1. Authored-by: vinodkc Signed-off-by: Jungtaek Lim --- .../tutorial/sql/python_data_source.rst | 9 ++- python/pyspark/errors/error-conditions.json | 5 ++ python/pyspark/sql/datasource_internal.py | 28 +++++++++- .../tests/test_python_streaming_datasource.py | 56 +++++++++++++++++++ 4 files changed, 95 insertions(+), 3 deletions(-) diff --git a/python/docs/source/tutorial/sql/python_data_source.rst b/python/docs/source/tutorial/sql/python_data_source.rst index 78ffeda0db1ce..1cc1811000ca4 100644 --- a/python/docs/source/tutorial/sql/python_data_source.rst +++ b/python/docs/source/tutorial/sql/python_data_source.rst @@ -305,7 +305,14 @@ This is the same dummy streaming reader that generate 2 rows every batch impleme def read(self, start: dict) -> Tuple[Iterator[Tuple], dict]: """ - Takes start offset as an input, return an iterator of tuples and the start offset of next read. + Takes start offset as an input, return an iterator of tuples and + the end offset (start offset for the next read). The end offset must + advance past the start offset when returning data; otherwise Spark + raises a validation exception. + For example, returning 2 records from start_idx 0 means end should + be {"offset": 2} (i.e. start + 2). + When there is no data to read, you may return the same offset as end and + start, but you must provide an empty iterator. """ start_idx = start["offset"] it = iter([(i,) for i in range(start_idx, start_idx + 2)]) diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index 295b372cade5c..326671c0d5ad2 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1119,6 +1119,11 @@ "SparkContext or SparkSession should be created first." ] }, + "SIMPLE_STREAM_READER_OFFSET_DID_NOT_ADVANCE": { + "message": [ + "SimpleDataSourceStreamReader.read() returned a non-empty batch but the end offset: did not advance past the start offset: . The end offset must represent the position after the last record returned." + ] + }, "SLICE_WITH_STEP": { "message": [ "Slice with step is not supported." diff --git a/python/pyspark/sql/datasource_internal.py b/python/pyspark/sql/datasource_internal.py index 6df0be4192ec8..9467bfdf73bc2 100644 --- a/python/pyspark/sql/datasource_internal.py +++ b/python/pyspark/sql/datasource_internal.py @@ -28,7 +28,7 @@ SimpleDataSourceStreamReader, ) from pyspark.sql.types import StructType -from pyspark.errors import PySparkNotImplementedError +from pyspark.errors import PySparkException, PySparkNotImplementedError def _streamReader(datasource: DataSource, schema: StructType) -> "DataSourceStreamReader": @@ -88,12 +88,36 @@ def initialOffset(self) -> dict: self.initial_offset = self.simple_reader.initialOffset() return self.initial_offset + def add_result_to_cache(self, start: dict, end: dict, it: Iterator[Tuple]) -> None: + """ + Validates that read() did not return a non-empty batch with end equal to start, + which would cause the same batch to be processed repeatedly. When end != start, + appends the result to the cache; when end == start with empty iterator, does not + cache (avoids unbounded cache growth). + """ + start_str = json.dumps(start) + end_str = json.dumps(end) + if end_str != start_str: + self.cache.append(PrefetchedCacheEntry(start, end, it)) + return + try: + next(it) + except StopIteration: + return + raise PySparkException( + errorClass="SIMPLE_STREAM_READER_OFFSET_DID_NOT_ADVANCE", + messageParameters={ + "start_offset": start_str, + "end_offset": end_str, + }, + ) + def latestOffset(self) -> dict: # when query start for the first time, use initial offset as the start offset. if self.current_offset is None: self.current_offset = self.initialOffset() (iter, end) = self.simple_reader.read(self.current_offset) - self.cache.append(PrefetchedCacheEntry(self.current_offset, end, iter)) + self.add_result_to_cache(self.current_offset, end, iter) self.current_offset = end return end diff --git a/python/pyspark/sql/tests/test_python_streaming_datasource.py b/python/pyspark/sql/tests/test_python_streaming_datasource.py index 9879231540f1d..ecf28677689b2 100644 --- a/python/pyspark/sql/tests/test_python_streaming_datasource.py +++ b/python/pyspark/sql/tests/test_python_streaming_datasource.py @@ -34,6 +34,7 @@ have_pyarrow, pyarrow_requirement_message, ) +from pyspark.errors import PySparkException from pyspark.testing import assertDataFrameEqual from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -251,6 +252,61 @@ def check_batch(df, batch_id): q.awaitTermination() self.assertIsNone(q.exception(), "No exception has to be propagated.") + def test_simple_stream_reader_offset_did_not_advance_raises(self): + """Returning end == start with non-empty data raises + SIMPLE_STREAM_READER_OFFSET_DID_NOT_ADVANCE.""" + from pyspark.sql.datasource_internal import _SimpleStreamReaderWrapper + + class BuggySimpleStreamReader(SimpleDataSourceStreamReader): + def initialOffset(self): + return {"offset": 0} + + def read(self, start: dict): + # Bug: return same offset as end despite returning data + start_idx = start["offset"] + it = iter([(i,) for i in range(start_idx, start_idx + 3)]) + return (it, start) + + def readBetweenOffsets(self, start: dict, end: dict): + return iter([]) + + def commit(self, end: dict): + pass + + reader = BuggySimpleStreamReader() + wrapper = _SimpleStreamReaderWrapper(reader) + with self.assertRaises(PySparkException) as cm: + wrapper.latestOffset() + self.assertEqual( + cm.exception.getCondition(), + "SIMPLE_STREAM_READER_OFFSET_DID_NOT_ADVANCE", + ) + + def test_simple_stream_reader_empty_iterator_start_equals_end_allowed(self): + """read() with end == start and empty iterator: no exception, no cache entry.""" + from pyspark.sql.datasource_internal import _SimpleStreamReaderWrapper + + class EmptyBatchReader(SimpleDataSourceStreamReader): + def initialOffset(self): + return {"offset": 0} + + def read(self, start: dict): + # Valid: same offset as end but empty iterator (no data) + return (iter([]), start) + + def readBetweenOffsets(self, start: dict, end: dict): + return iter([]) + + def commit(self, end: dict): + pass + + reader = EmptyBatchReader() + wrapper = _SimpleStreamReaderWrapper(reader) + end = wrapper.latestOffset() + start = {"offset": 0} + self.assertEqual(end, start) + self.assertEqual(len(wrapper.cache), 0) + def test_stream_writer(self): input_dir = tempfile.TemporaryDirectory(prefix="test_data_stream_write_input") output_dir = tempfile.TemporaryDirectory(prefix="test_data_stream_write_output") From 52f7dd7cad30938776f1ed7f164de63f69e59058 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Fri, 20 Feb 2026 14:00:44 -0800 Subject: [PATCH 385/400] [SPARK-55302][SQL][4.1] Fix custom metrics in case of `KeyGroupedPartitioning` ### What changes were proposed in this pull request? This PR adds a new `initMetricsValues()` method to `PartitionReader` so as to initialize custom metrics returned by `currentMetricsValues()`. In case of `KeyGroupedPartitioning` multiple input partitions are grouped and so multiple `PartitionReader` belong to one output partition. A `PartitionReader` needs to be initialized with metrics calculated by the previous `PartitionReader` of the same partition group to calculate the right value. ### Why are the changes needed? To calculate custom metrics correctly. ### Does this PR introduce _any_ user-facing change? It fixes metrics calculation. ### How was this patch tested? New UT is added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54401 from peter-toth/SPARK-55302-fix-kgp-custom-metrics-4.1. Authored-by: Peter Toth Signed-off-by: Dongjoon Hyun --- .../sql/connector/read/PartitionReader.java | 9 ++++++ .../connector/catalog/InMemoryBaseTable.scala | 30 ++++++++++++++++++- .../datasources/v2/DataSourceRDD.scala | 21 +++++++++---- .../KeyGroupedPartitioningSuite.scala | 18 +++++++++++ .../InMemoryTableMetricSuite.scala | 22 +------------- .../spark/sql/test/SharedSparkSession.scala | 23 ++++++++++++++ 6 files changed, 95 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java index 5286bbf9f85a1..c12bc14a49c44 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java @@ -58,4 +58,13 @@ default CustomTaskMetric[] currentMetricsValues() { CustomTaskMetric[] NO_METRICS = {}; return NO_METRICS; } + + /** + * Sets the initial value of metrics before fetching any data from the reader. This is called + * when multiple {@link PartitionReader}s are grouped into one partition in case of + * {@link org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning} and the reader + * is initialized with the metrics returned by the previous reader that belongs to the same + * partition. By default, this method does nothing. + */ + default void initMetricsValues(CustomTaskMetric[] metrics) {} } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala index 29b336c780f8d..407d592f82199 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala @@ -554,6 +554,10 @@ abstract class InMemoryBaseTable( } new BufferedRowsReaderFactory(metadataColumns.toSeq, nonMetadataColumns, tableSchema) } + + override def supportedCustomMetrics(): Array[CustomMetric] = { + Array(new RowsReadCustomMetric) + } } case class InMemoryBatchScan( @@ -841,10 +845,13 @@ private class BufferedRowsReader( } private var index: Int = -1 + private var rowsRead: Long = 0 override def next(): Boolean = { index += 1 - index < partition.rows.length + val hasNext = index < partition.rows.length + if (hasNext) rowsRead += 1 + hasNext } override def get(): InternalRow = { @@ -987,6 +994,22 @@ private class BufferedRowsReader( private def castElement(elem: Any, toType: DataType, fromType: DataType): Any = Cast(Literal(elem, fromType), toType, None, EvalMode.TRY).eval(null) + + override def initMetricsValues(metrics: Array[CustomTaskMetric]): Unit = { + metrics.foreach { m => + m.name match { + case "rows_read" => rowsRead = m.value() + } + } + } + + override def currentMetricsValues(): Array[CustomTaskMetric] = { + val metric = new CustomTaskMetric { + override def name(): String = "rows_read" + override def value(): Long = rowsRead + } + Array(metric) + } } private class BufferedRowsWriterFactory(schema: StructType) @@ -1055,6 +1078,11 @@ class InMemoryCustomDriverTaskMetric(value: Long) extends CustomTaskMetric { override def value(): Long = value } +class RowsReadCustomMetric extends CustomSumMetric { + override def name(): String = "rows_read" + override def description(): String = "number of rows read" +} + case class Commit(id: Long, writeSummary: Option[WriteSummary] = None) sealed trait Operation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala index 6a07d3c3931a1..fbf5c06fe051b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala @@ -24,6 +24,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.metric.CustomTaskMetric import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric} @@ -97,7 +98,8 @@ class DataSourceRDD( } // Once we advance to the next partition, update the metric callback for early finish - partitionMetricCallback.advancePartition(iter, reader) + val previousMetrics = partitionMetricCallback.advancePartition(iter, reader) + previousMetrics.foreach(reader.initMetricsValues) currentIter = Some(iter) hasNext @@ -118,19 +120,26 @@ private class PartitionMetricCallback private var iter: MetricsIterator[_] = null private var reader: PartitionReader[_] = null - def advancePartition(iter: MetricsIterator[_], reader: PartitionReader[_]): Unit = { - execute() + def advancePartition( + iter: MetricsIterator[_], + reader: PartitionReader[_]): Option[Array[CustomTaskMetric]] = { + val metrics = execute() this.iter = iter this.reader = reader + + metrics } - def execute(): Unit = { + def execute(): Option[Array[CustomTaskMetric]] = { if (iter != null && reader != null) { - CustomMetrics - .updateMetrics(reader.currentMetricsValues.toImmutableArraySeq, customMetrics) + val metrics = reader.currentMetricsValues + CustomMetrics.updateMetrics(metrics.toImmutableArraySeq, customMetrics) iter.forceUpdateMetrics() reader.close() + Some(metrics) + } else { + None } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 274741fb8e7be..409e3fa922407 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -2822,6 +2822,24 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { } } + test("SPARK-55302: Custom metrics of grouped partitions") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(4, 'bb', 10.0, cast('2021-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2021-02-01' as timestamp))") + + val metrics = runAndFetchMetrics { + val df = sql(s"SELECT * FROM testcat.ns.$items") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans(0).inputRDD.partitions.length === 2, "items scan should have 2 partition groups") + df.collect() + } + assert(metrics("number of rows read") == "3") + } + test("SPARK-55411: Fix ArrayIndexOutOfBoundsException when join keys " + "are less than cluster keys") { withSQLConf( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala index 7e8a95f4d0cdf..502424d58d2cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.datasources import java.util.Collections import org.scalatest.BeforeAndAfter -import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.QueryTest import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTable, InMemoryTableCatalog} @@ -54,27 +53,8 @@ class InMemoryTableMetricSuite Array(Column.create("i", IntegerType)), Array.empty[Transform], Collections.emptyMap[String, String]) - func("testcat.table_name") + val metrics = runAndFetchMetrics(func("testcat.table_name")) - // Wait until the new execution is started and being tracked. - eventually(timeout(10.seconds), interval(10.milliseconds)) { - assert(statusStore.executionsCount() >= oldCount) - } - - // Wait for listener to finish computing the metrics for the execution. - eventually(timeout(10.seconds), interval(10.milliseconds)) { - assert(statusStore.executionsList().nonEmpty && - statusStore.executionsList().last.metricValues != null) - } - - val exec = statusStore.executionsList().last - val execId = exec.executionId - val sqlMetrics = exec.metrics.map { metric => - metric.accumulatorId -> metric.name - }.toMap - val metrics = statusStore.executionMetrics(execId).map { case (k, v) => - sqlMetrics(k) -> v - } checker(metrics) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index 245219c1756d9..720b13b812e08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -54,6 +54,29 @@ trait SharedSparkSession extends SQLTestUtils with SharedSparkSessionBase { doThreadPostAudit() } } + + def runAndFetchMetrics(func: => Unit): Map[String, String] = { + val statusStore = spark.sharedState.statusStore + val oldCount = statusStore.executionsList().size + + func + + // Wait until the new execution is started and being tracked. + eventually(timeout(10.seconds), interval(10.milliseconds)) { + assert(statusStore.executionsCount() >= oldCount) + } + + // Wait for listener to finish computing the metrics for the execution. + eventually(timeout(10.seconds), interval(10.milliseconds)) { + assert(statusStore.executionsList().nonEmpty && + statusStore.executionsList().last.metricValues != null) + } + + val exec = statusStore.executionsList().last + val execId = exec.executionId + val sqlMetrics = exec.metrics.map { metric => metric.accumulatorId -> metric.name }.toMap + statusStore.executionMetrics(execId).map { case (k, v) => sqlMetrics(k) -> v } + } } /** From b119334b8439a21eff32819beb81749c0488f52c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 21 Feb 2026 16:27:03 -0800 Subject: [PATCH 386/400] [SPARK-55619][SQL][4.1] Fix custom metrics in case of coalesced partitions ### What changes were proposed in this pull request? Replace `PartitionMetricCallback` with a `ConcurrentHashMap` keyed by task attempt ID to correctly track reader state across multiple `compute()` calls when `DataSourceRDD` is coalesced. The completion listener is registered only once per task attempt, and metrics are flushed and carried forward between readers as partitions are advanced. ### Why are the changes needed? When `DataSourceRDD` is coalesced (e.g., via `.coalesce(1)`), `compute()` gets called multiple times per task, which causes the custom metrics incorrect. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Sonnet 4.6 Closes #54407 from viirya/SPARK-55619-branch-4.1. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../datasources/v2/DataSourceRDD.scala | 95 +++++++++++-------- .../KeyGroupedPartitioningSuite.scala | 15 +++ 2 files changed, 69 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala index fbf5c06fe051b..19a057c72506b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.v2 +import java.util.concurrent.ConcurrentHashMap + import scala.language.existentials import org.apache.spark._ @@ -24,7 +26,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.metric.CustomTaskMetric import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric} @@ -34,6 +35,19 @@ import org.apache.spark.util.ArrayImplicits._ class DataSourceRDDPartition(val index: Int, val inputPartitions: Seq[InputPartition]) extends Partition with Serializable +/** + * Holds the state for a reader in a task, used by the completion listener to access the most + * recently created reader and iterator for final metrics updates and cleanup. + * + * When `compute()` is called multiple times for the same task (e.g., when DataSourceRDD is + * coalesced), this state is updated on each call to track the most recent reader. The task + * completion listener then uses this most recent reader for final cleanup and metrics reporting. + * + * @param reader The partition reader + * @param iterator The metrics iterator wrapping the reader + */ +private case class ReaderState(reader: PartitionReader[_], iterator: MetricsIterator[_]) + // TODO: we should have 2 RDDs: an RDD[InternalRow] for row-based scan, an `RDD[ColumnarBatch]` for // columnar scan. class DataSourceRDD( @@ -44,6 +58,11 @@ class DataSourceRDD( customMetrics: Map[String, SQLMetric]) extends RDD[InternalRow](sc, Nil) { + // Map from task attempt ID to the most recently created ReaderState for that task. + // When compute() is called multiple times for the same task (due to coalescing), the map entry + // is updated each time so the completion listener always closes the last reader. + @transient private lazy val taskReaderStates = new ConcurrentHashMap[Long, ReaderState]() + override protected def getPartitions: Array[Partition] = { inputPartitions.zipWithIndex.map { case (inputPartitions, index) => new DataSourceRDDPartition(index, inputPartitions) @@ -56,20 +75,34 @@ class DataSourceRDD( } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val taskAttemptId = context.taskAttemptId() + + // Add completion listener only once per task attempt. When compute() is called a second time + // for the same task (e.g., due to coalescing), the first call will have already put a + // ReaderState into taskReaderStates, so containsKey returns true and we skip this block. + if (!taskReaderStates.containsKey(taskAttemptId)) { + context.addTaskCompletionListener[Unit] { ctx => + // In case of early stopping before consuming the entire iterator, + // we need to do one more metric update at the end of the task. + try { + val readerState = taskReaderStates.get(ctx.taskAttemptId()) + if (readerState != null) { + CustomMetrics.updateMetrics( + readerState.reader.currentMetricsValues.toImmutableArraySeq, customMetrics) + readerState.iterator.forceUpdateMetrics() + readerState.reader.close() + } + } finally { + taskReaderStates.remove(ctx.taskAttemptId()) + } + } + } val iterator = new Iterator[Object] { private val inputPartitions = castPartition(split).inputPartitions private var currentIter: Option[Iterator[Object]] = None private var currentIndex: Int = 0 - private val partitionMetricCallback = new PartitionMetricCallback(customMetrics) - - // In case of early stopping before consuming the entire iterator, - // we need to do one more metric update at the end of the task. - context.addTaskCompletionListener[Unit] { _ => - partitionMetricCallback.execute() - } - override def hasNext: Boolean = currentIter.exists(_.hasNext) || advanceToNextIter() override def next(): Object = { @@ -97,9 +130,18 @@ class DataSourceRDD( (iter, rowReader) } - // Once we advance to the next partition, update the metric callback for early finish - val previousMetrics = partitionMetricCallback.advancePartition(iter, reader) - previousMetrics.foreach(reader.initMetricsValues) + // Flush metrics and close the previous reader before advancing to the next one. + // Pass the accumulated metrics to the new reader so they carry forward correctly. + val prevState = taskReaderStates.get(taskAttemptId) + if (prevState != null) { + val metrics = prevState.reader.currentMetricsValues + CustomMetrics.updateMetrics(metrics.toImmutableArraySeq, customMetrics) + reader.initMetricsValues(metrics) + prevState.reader.close() + } + + // Update the map so the completion listener always references the latest reader. + taskReaderStates.put(taskAttemptId, ReaderState(reader, iter)) currentIter = Some(iter) hasNext @@ -115,35 +157,6 @@ class DataSourceRDD( } } -private class PartitionMetricCallback - (customMetrics: Map[String, SQLMetric]) { - private var iter: MetricsIterator[_] = null - private var reader: PartitionReader[_] = null - - def advancePartition( - iter: MetricsIterator[_], - reader: PartitionReader[_]): Option[Array[CustomTaskMetric]] = { - val metrics = execute() - - this.iter = iter - this.reader = reader - - metrics - } - - def execute(): Option[Array[CustomTaskMetric]] = { - if (iter != null && reader != null) { - val metrics = reader.currentMetricsValues - CustomMetrics.updateMetrics(metrics.toImmutableArraySeq, customMetrics) - iter.forceUpdateMetrics() - reader.close() - Some(metrics) - } else { - None - } - } -} - private class PartitionIterator[T]( reader: PartitionReader[T], customMetrics: Map[String, SQLMetric]) extends Iterator[T] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 409e3fa922407..8a65cb623f6e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -2840,6 +2840,21 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { assert(metrics("number of rows read") == "3") } + test("SPARK-55619: Custom metrics of coalesced partitions") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql(s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(2, 'bb', 10.0, cast('2021-01-01' as timestamp))") + + val metrics = runAndFetchMetrics { + val df = sql(s"SELECT * FROM testcat.ns.$items").coalesce(1) + df.collect() + } + assert(metrics("number of rows read") == "2") + } + test("SPARK-55411: Fix ArrayIndexOutOfBoundsException when join keys " + "are less than cluster keys") { withSQLConf( From 27b7268e0edeee18521906aa4181cb55746708cb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 23 Feb 2026 07:15:37 -0800 Subject: [PATCH 387/400] [SPARK-55637][SQL][TESTS] Generalize `postgres-krb-setup.sh` to find config files ### What changes were proposed in this pull request? This PR aims to generalize `postgres-krb-setup.sh` to find config files instead of using static locations. ### Why are the changes needed? To improve `PostgresKrbIntegrationSuite` to handle different PostgreSQL image via `POSTGRES_DOCKER_IMAGE_NAME`. Currently, `PostgresKrbIntegrationSuite` fails for other `PostgreSQL` images. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: `Gemini 3.1 Pro (High)` on `Antigravity` Closes #54423 from dongjoon-hyun/SPARK-55637. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 920a401f8e027876d56856cc4ca58786d03c2add) Signed-off-by: Dongjoon Hyun --- .../src/test/resources/postgres-krb-setup.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connector/docker-integration-tests/src/test/resources/postgres-krb-setup.sh b/connector/docker-integration-tests/src/test/resources/postgres-krb-setup.sh index dd9fd8cb51adf..3c6aff0ee3227 100755 --- a/connector/docker-integration-tests/src/test/resources/postgres-krb-setup.sh +++ b/connector/docker-integration-tests/src/test/resources/postgres-krb-setup.sh @@ -16,6 +16,6 @@ # limitations under the License. # -sed -i 's/host all all all .*/host all all all gss/g' /var/lib/postgresql/data/pg_hba.conf -echo "krb_server_keyfile='/docker-entrypoint-initdb.d/postgres.keytab'" >> /var/lib/postgresql/data/postgresql.conf +sed -i 's/host all all all .*/host all all all gss/g' $(find /var/lib/postgresql -name pg_hba.conf) +echo "krb_server_keyfile='/docker-entrypoint-initdb.d/postgres.keytab'" >> $(find /var/lib/postgresql -name postgresql.conf) psql -U postgres -c "CREATE ROLE \"postgres/__IP_ADDRESS_REPLACE_ME__@EXAMPLE.COM\" LOGIN SUPERUSER" From bc1bbe6eb06eaf2faab730743b61b3a848a49c3f Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Mon, 23 Feb 2026 11:44:40 -0800 Subject: [PATCH 388/400] [SPARK-55525][PYTHON] Fix UDTF_ARROW_TYPE_CONVERSION_ERROR with undefined error message parameter ### What changes were proposed in this pull request? There is a mismatch between the error class definition for UDTF_ARROW_TYPE_CONVERSION_ERROR and its usage. The code in python/pyspark/worker.py was raising UDTF_ARROW_TYPE_CONVERSION_ERROR with parameters data, schema, and arrow_schema, but the error class definition in error-conditions.json did not expect any parameters. This PR fixed this by introducing a new error class UDTF_ARROW_DATA_CONVERSION_ERROR specifically for data conversion errors, which accepts the necessary parameters. ### Why are the changes needed? Fixing a bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? uT ### Was this patch authored or co-authored using generative AI tooling? Yes Closes #54318 from allisonwang-db/fix-udtf-error. Authored-by: Allison Wang Signed-off-by: Allison Wang (cherry picked from commit e7de36212cb109c271d6b4018760a2757886935a) Signed-off-by: Allison Wang --- python/pyspark/errors/error-conditions.json | 5 +++++ .../pyspark/sql/tests/arrow/test_arrow_udtf.py | 17 +++++++++++++++++ python/pyspark/sql/tests/test_udtf.py | 16 ++++++++-------- python/pyspark/worker.py | 2 +- 4 files changed, 31 insertions(+), 9 deletions(-) diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index 326671c0d5ad2..da8a9f1fea341 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1187,6 +1187,11 @@ "Return type of the user-defined function should be , but is ." ] }, + "UDTF_ARROW_DATA_CONVERSION_ERROR": { + "message": [ + "Cannot convert UDTF output to Arrow. Data: . Schema: . Arrow Schema: ." + ] + }, "UDTF_ARROW_TYPE_CAST_ERROR": { "message": [ "Cannot convert the output value of the column '' with type '' to the specified return type of the column: ''. Please check if the data types match and try again." diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py index cc0edda378abf..2be399171abb8 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py @@ -33,6 +33,23 @@ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) class ArrowUDTFTestsMixin: + def test_arrow_udtf_data_conversion_error(self): + from pyspark.sql.functions import udtf + + @udtf(returnType="x int, y int") + class DataConversionErrorUDTF: + def eval(self): + # Return a non-tuple value when multiple return values are expected. + # This will cause LocalDataToArrowConversion.convert to fail with TypeError (len() on int), + # which should be wrapped in UDTF_ARROW_DATA_CONVERSION_ERROR. + yield 1 + + # Enable Arrow optimization for regular UDTFs + with self.sql_conf({"spark.sql.execution.pythonUDTF.arrow.enabled": "true"}): + with self.assertRaisesRegex(PythonException, "UDTF_ARROW_DATA_CONVERSION_ERROR"): + result_df = DataConversionErrorUDTF() + result_df.collect() + def test_arrow_udtf_zero_args(self): @arrow_udtf(returnType="id int, value string") class TestUDTF: diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 5ded5aa67b4eb..54bd2a34ff1de 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3548,7 +3548,7 @@ class TestUDTF: def eval(self): yield 1, - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), @@ -3575,7 +3575,7 @@ class TestUDTF: def eval(self): yield "1", - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), @@ -3604,7 +3604,7 @@ class TestUDTF: def eval(self): yield "hello", - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3632,7 +3632,7 @@ class TestUDTF: def eval(self): yield [0, 1.1, 2], - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3664,7 +3664,7 @@ class TestUDTF: def eval(self): yield {"a": 0, "b": 1.1, "c": 2}, - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3695,7 +3695,7 @@ class TestUDTF: def eval(self): yield {"a": 0, "b": 1.1, "c": 2}, - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3725,7 +3725,7 @@ class TestUDTF: def eval(self): yield Row(a=0, b=1.1, c=2), - err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" + err = "UDTF_ARROW_DATA_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3761,7 +3761,7 @@ def eval(self): "x: array", ]: with self.subTest(ret_type=ret_type): - with self.assertRaisesRegex(PythonException, "UDTF_ARROW_TYPE_CONVERSION_ERROR"): + with self.assertRaisesRegex(PythonException, "UDTF_ARROW_DATA_CONVERSION_ERROR"): udtf(TestUDTF, returnType=ret_type)().collect() def test_decimal_round(self): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4bae9f6dc48f5..fe68b8baa6c90 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -2342,7 +2342,7 @@ def convert_to_arrow(data: Iterable): def raise_conversion_error(original_exception): raise PySparkRuntimeError( - errorClass="UDTF_ARROW_TYPE_CONVERSION_ERROR", + errorClass="UDTF_ARROW_DATA_CONVERSION_ERROR", messageParameters={ "data": str(data), "schema": return_type.simpleString(), From 757cc18251c5a364fdd2bd369206d50bbe69bbd2 Mon Sep 17 00:00:00 2001 From: Yan Yan Date: Mon, 23 Feb 2026 15:52:03 -0800 Subject: [PATCH 389/400] [SPARK-51834][SQL][FOLLOW-UP][4.1] Add missing `.withConstraints()` in `AtomicReplaceTableExec` (This PR is to backport https://github.com/apache/spark/pull/54322) ### What changes were proposed in this pull request? SPARK-51834 added `.withConstraints()` to `CreateTableExec` and `ReplaceTableExec` but missed `AtomicReplaceTableExec` in the same file. This causes `REPLACE TABLE` and `CREATE OR REPLACE TABLE` with constraints to silently drop them when the catalog implements `StagingTableCatalog`. Also fixes `StagingInMemoryTableCatalog` to forward constraints when constructing `InMemoryTable`, and adds regression tests for all four constraint types through the atomic replace path. ### Why are the changes needed? `AtomicReplaceTableExec` does not call `.withConstraints()` when building `TableInfo`, so `REPLACE TABLE` and `CREATE OR REPLACE TABLE` silently drop constraints when the catalog implements `StagingTableCatalog`. ### Does this PR introduce _any_ user-facing change? Yes. Previously, REPLACE TABLE and CREATE OR REPLACE TABLE with constraints would silently drop constraints when using a `StagingTableCatalog`. After this fix, constraints are correctly passed through to the catalog. ### How was this patch tested? New unit tests in `UniqueConstraintSuite`, `PrimaryKeyConstraintSuite`, `CheckConstraintSuite`, and `ForeignKeyConstraintSuite` that exercise `REPLACE TABLE` with constraints through the atomic catalog path (`StagingInMemoryTableCatalog`). Verified that all 4 new tests fail without the fix and pass with it. All existing tests in these suites continue to pass. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Claude Opus 4.6 Closes #54322 from yyanyy/staging_catalog_constraint. Authored-by: Yan Yan Closes #54404 from yyanyy/backport-SPARK-51834-follow-up-branch-4.1. Authored-by: Yan Yan Signed-off-by: Gengliang Wang --- .../catalog/StagingInMemoryTableCatalog.scala | 9 ++++++--- .../datasources/v2/ReplaceTableExec.scala | 2 ++ .../command/v2/CheckConstraintSuite.scala | 14 ++++++++++++++ .../command/v2/CommandSuiteBase.scala | 2 ++ .../command/v2/ForeignKeyConstraintSuite.scala | 18 ++++++++++++++++++ .../command/v2/PrimaryKeyConstraintSuite.scala | 15 +++++++++++++++ .../command/v2/UniqueConstraintSuite.scala | 15 +++++++++++++++ 7 files changed, 72 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala index ee2400cab35c8..7ded99c709a39 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala @@ -36,7 +36,8 @@ class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTable new TestStagedCreateTable( ident, new InMemoryTable(s"$name.${ident.quoted}", - tableInfo.schema(), tableInfo.partitions(), tableInfo.properties())) + tableInfo.columns(), tableInfo.partitions(), tableInfo.properties(), + tableInfo.constraints())) } override def stageReplace(ident: Identifier, tableInfo: TableInfo): StagedTable = { @@ -44,7 +45,8 @@ class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTable new TestStagedReplaceTable( ident, new InMemoryTable(s"$name.${ident.quoted}", - tableInfo.schema(), tableInfo.partitions(), tableInfo.properties())) + tableInfo.columns(), tableInfo.partitions(), tableInfo.properties(), + tableInfo.constraints())) } override def stageCreateOrReplace(ident: Identifier, tableInfo: TableInfo) : StagedTable = { @@ -52,7 +54,8 @@ class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTable new TestStagedCreateOrReplaceTable( ident, new InMemoryTable(s"$name.${ident.quoted}", - tableInfo.schema(), tableInfo.partitions(), tableInfo.properties)) + tableInfo.columns(), tableInfo.partitions(), tableInfo.properties(), + tableInfo.constraints())) } private def validateStagedTable( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 7ce95ced0d242..454a4041d36e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -83,6 +83,7 @@ case class AtomicReplaceTableExec( .withColumns(columns) .withPartitions(partitioning.toArray) .withProperties(tableProperties.asJava) + .withConstraints(tableSpec.constraints.toArray) .build() catalog.stageCreateOrReplace(identifier, tableInfo) } else if (catalog.tableExists(identifier)) { @@ -91,6 +92,7 @@ case class AtomicReplaceTableExec( .withColumns(columns) .withPartitions(partitioning.toArray) .withProperties(tableProperties.asJava) + .withConstraints(tableSpec.constraints.toArray) .build() catalog.stageReplace(identifier, tableInfo) } catch { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index ee2dd476958e1..8e295ecc3d5de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -205,6 +205,20 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } } + test("Replace table with check constraint using atomic catalog") { + getConstraintCharacteristics().foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", atomicCatalog) { t => + val constraintStr = s"CONSTRAINT c1 CHECK (id > 0) $characteristic" + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"REPLACE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(atomicCatalog, "ns", "tbl") + val constraint = getCheckConstraint(table) + assert(constraint.name() == "c1") + assert(constraint.toDDL == s"CONSTRAINT c1 CHECK (id > 0) $expectedDDL") + } + } + } + test("Alter table add check constraint") { getConstraintCharacteristics().foreach { case (characteristic, expectedDDL) => withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala index 24bc4483d31c7..548e15f9e783b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala @@ -35,6 +35,7 @@ trait CommandSuiteBase extends SharedSparkSession { def commandVersion: String = "V2" // The command version is added to test names def catalog: String = "test_catalog" // The default V2 catalog for testing def nonPartitionCatalog: String = "non_part_test_catalog" // Catalog for non-partitioned tables + def atomicCatalog: String = "atomic_test_catalog" // Catalog with StagingTableCatalog support def rowLevelOPCatalog: String = "row_level_op_catalog" def defaultUsing: String = "USING _" // The clause is used in creating v2 tables under testing @@ -42,6 +43,7 @@ trait CommandSuiteBase extends SharedSparkSession { override def sparkConf: SparkConf = super.sparkConf .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) .set(s"spark.sql.catalog.$nonPartitionCatalog", classOf[InMemoryTableCatalog].getName) + .set(s"spark.sql.catalog.$atomicCatalog", classOf[StagingInMemoryTableCatalog].getName) .set(s"spark.sql.catalog.fun_$catalog", classOf[InMemoryCatalog].getName) .set(s"spark.sql.catalog.$rowLevelOPCatalog", classOf[InMemoryRowLevelOperationTableCatalog].getName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala index a876013490ea7..b1eb85b450639 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala @@ -85,6 +85,24 @@ class ForeignKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL } } + test("REPLACE table with foreign key constraint using atomic catalog") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", atomicCatalog) { t => + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"CREATE TABLE ${t}_ref (id bigint, data string) $defaultUsing") + val constraintStr = s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES ${t}_ref(id) $characteristic" + sql(s"REPLACE TABLE $t (id bigint, fk bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(atomicCatalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "fk1") + assert(constraint.toDDL == s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES $atomicCatalog.ns.tbl_ref (id) $expectedDDL") + } + } + } + test("Add duplicated foreign key constraint") { withNamespaceAndTable("ns", "tbl", catalog) { t => sql(s"CREATE TABLE $t (id bigint, fk bigint, data string) $defaultUsing") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala index f692f9588161e..f7d551d4e34d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala @@ -76,6 +76,21 @@ class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL } } + test("Replace table with primary key constraint using atomic catalog") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", atomicCatalog) { t => + val constraintStr = s"CONSTRAINT pk1 PRIMARY KEY (id) $characteristic" + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"REPLACE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(atomicCatalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "pk1") + assert(constraint.toDDL == s"CONSTRAINT pk1 PRIMARY KEY (id) $expectedDDL") + } + } + } + test("Add duplicated primary key constraint") { withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala index 6efc3912af9d2..96a9945b10f9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala @@ -76,6 +76,21 @@ class UniqueConstraintSuite extends QueryTest with CommandSuiteBase with DDLComm } } + test("Replace table with unique constraint using atomic catalog") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", atomicCatalog) { t => + val constraintStr = s"CONSTRAINT uk1 UNIQUE (id) $characteristic" + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"REPLACE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(atomicCatalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "uk1") + assert(constraint.toDDL == s"CONSTRAINT uk1 UNIQUE (id) $expectedDDL") + } + } + } + test("Add duplicated unique constraint") { withNamespaceAndTable("ns", "tbl", catalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") From 91ed66854e076198f22eaa2d84112cc42345becc Mon Sep 17 00:00:00 2001 From: ilicmarkodb Date: Tue, 24 Feb 2026 13:24:27 +0800 Subject: [PATCH 390/400] [SPARK-55647][SQL] Fix `ConstantPropagation` incorrectly replacing attributes with non-binary-stable collations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? * `ConstantPropagation` optimizer rule substitutes attributes with literals derived from equality predicates (e.g. `c = 'hello'`), then propagates them into other conditions in the same conjunction. This is unsafe for non-binary-stable collations (e.g. `UTF8_LCASE`) where equality is non-identity: `c = 'hello'` (case-insensitive) does not imply `c` holds exactly the bytes `'hello'` - it could also be `'HELLO'`, `'Hello'`, etc. * Substituting `c → 'hello'` in a second condition like `c = 'HELLO' COLLATE UNICODE` turns it into the constant expression `'hello' = 'HELLO' COLLATE UNICODE`, which is always `false`, producing incorrect results. * Fixed by guarding `safeToReplace` with `isBinaryStable(ar.dataType)` so propagation is skipped for attributes whose type is not binary-stable. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54435 from ilicmarkodb/fix_collation_. Authored-by: ilicmarkodb Signed-off-by: Wenchen Fan (cherry picked from commit ec35791b3d987fed2f7b2a872fbbbaf30ada20d5) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/optimizer/expressions.scala | 12 +++++++++++- .../apache/spark/sql/collation/CollationSuite.scala | 13 +++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 71eb3e5ea2bd7..c0fc8d0bae42b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.{AlwaysProcess, TreeNodeTag} import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.util.CharVarcharUtils.CHAR_VARCHAR_TYPE_STRING_METADATA_KEY +import org.apache.spark.sql.catalyst.util.UnsafeRowUtils.isBinaryStable import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -217,8 +218,17 @@ object ConstantPropagation extends Rule[LogicalPlan] { // substituted into `1 + 1 = 1` if 'c' isn't nullable. If 'c' is nullable then the enclosing // NOT prevents us to do the substitution as NOT flips the context (`nullIsFalse`) of what a // null result of the enclosed expression means. + // + // Also, we shouldn't replace attributes with non-binary-stable data types, since this can lead + // to incorrect results. For example: + // `CREATE TABLE t (c STRING COLLATE UTF8_LCASE);` + // `INSERT INTO t VALUES ('HELLO'), ('hello');` + // `SELECT * FROM t WHERE c = 'hello' AND c = 'HELLO' COLLATE UNICODE;` + // If we replace `c` with `'hello'`, we get `'hello' = 'HELLO' COLLATE UNICODE` for the right + // condition, which is false, while the original `c = 'HELLO' COLLATE UNICODE` is true for + // 'HELLO' and false for 'hello'. private def safeToReplace(ar: AttributeReference, nullIsFalse: Boolean) = - !ar.nullable || nullIsFalse + (!ar.nullable || nullIsFalse) && isBinaryStable(ar.dataType) private def replaceConstants( condition: Expression, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala index c84647066f25d..8f7a68bcbe6f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala @@ -2195,4 +2195,17 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { Seq()) } } + + test("ConstantPropagation does not replace attributes with non-binary-stable collation") { + val tableName = "t1" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (c STRING COLLATE UTF8_LCASE)") + sql(s"INSERT INTO $tableName VALUES ('hello'), ('HELLO')") + + checkAnswer( + sql(s"SELECT * FROM $tableName WHERE c = 'hello' AND c = 'HELLO' COLLATE UNICODE"), + Row("HELLO") + ) + } + } } From 43866aa0b10373e294b140a575c55365af64c963 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 25 Feb 2026 10:21:02 -0800 Subject: [PATCH 391/400] [SPARK-55688][BUILD] Upgrade `aircompressor` to 2.0.3 ### What changes were proposed in this pull request? This PR aims to upgrade `aircompressor` to 2.0.3. ### Why are the changes needed? To bring the latest bug fixes. - https://github.com/airlift/aircompressor/releases/tag/2.0.3 ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #54486 from dongjoon-hyun/SPARK-55688. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 0a7908e95585b1acb154ca7228e6887082fa2a86) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 327e0a8b0b6a8..3720185bab318 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -3,7 +3,7 @@ JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar RoaringBitmap/1.3.0//RoaringBitmap-1.3.0.jar ST4/4.0.4//ST4-4.0.4.jar -aircompressor/2.0.2//aircompressor-2.0.2.jar +aircompressor/2.0.3//aircompressor-2.0.3.jar algebra_2.13/2.8.0//algebra_2.13-2.8.0.jar aliyun-java-sdk-core/4.5.10//aliyun-java-sdk-core-4.5.10.jar aliyun-java-sdk-kms/2.11.0//aliyun-java-sdk-kms-2.11.0.jar diff --git a/pom.xml b/pom.xml index 1d76def7d2fe8..c48218cb2e74d 100644 --- a/pom.xml +++ b/pom.xml @@ -2338,7 +2338,7 @@ io.airlift aircompressor - 2.0.2 + 2.0.3 org.apache.orc From 4b1e6b2f95ef8436b3e075f297c53be7505ceaad Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 25 Feb 2026 10:27:18 -0800 Subject: [PATCH 392/400] Revert "[SPARK-55525][PYTHON] Fix UDTF_ARROW_TYPE_CONVERSION_ERROR with undefined error message parameter" This reverts commit bc1bbe6eb06eaf2faab730743b61b3a848a49c3f. --- python/pyspark/errors/error-conditions.json | 5 ----- .../pyspark/sql/tests/arrow/test_arrow_udtf.py | 17 ----------------- python/pyspark/sql/tests/test_udtf.py | 16 ++++++++-------- python/pyspark/worker.py | 2 +- 4 files changed, 9 insertions(+), 31 deletions(-) diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index da8a9f1fea341..326671c0d5ad2 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1187,11 +1187,6 @@ "Return type of the user-defined function should be , but is ." ] }, - "UDTF_ARROW_DATA_CONVERSION_ERROR": { - "message": [ - "Cannot convert UDTF output to Arrow. Data: . Schema: . Arrow Schema: ." - ] - }, "UDTF_ARROW_TYPE_CAST_ERROR": { "message": [ "Cannot convert the output value of the column '' with type '' to the specified return type of the column: ''. Please check if the data types match and try again." diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py index 2be399171abb8..cc0edda378abf 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py @@ -33,23 +33,6 @@ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) class ArrowUDTFTestsMixin: - def test_arrow_udtf_data_conversion_error(self): - from pyspark.sql.functions import udtf - - @udtf(returnType="x int, y int") - class DataConversionErrorUDTF: - def eval(self): - # Return a non-tuple value when multiple return values are expected. - # This will cause LocalDataToArrowConversion.convert to fail with TypeError (len() on int), - # which should be wrapped in UDTF_ARROW_DATA_CONVERSION_ERROR. - yield 1 - - # Enable Arrow optimization for regular UDTFs - with self.sql_conf({"spark.sql.execution.pythonUDTF.arrow.enabled": "true"}): - with self.assertRaisesRegex(PythonException, "UDTF_ARROW_DATA_CONVERSION_ERROR"): - result_df = DataConversionErrorUDTF() - result_df.collect() - def test_arrow_udtf_zero_args(self): @arrow_udtf(returnType="id int, value string") class TestUDTF: diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 54bd2a34ff1de..5ded5aa67b4eb 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3548,7 +3548,7 @@ class TestUDTF: def eval(self): yield 1, - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), @@ -3575,7 +3575,7 @@ class TestUDTF: def eval(self): yield "1", - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), @@ -3604,7 +3604,7 @@ class TestUDTF: def eval(self): yield "hello", - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3632,7 +3632,7 @@ class TestUDTF: def eval(self): yield [0, 1.1, 2], - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3664,7 +3664,7 @@ class TestUDTF: def eval(self): yield {"a": 0, "b": 1.1, "c": 2}, - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3695,7 +3695,7 @@ class TestUDTF: def eval(self): yield {"a": 0, "b": 1.1, "c": 2}, - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3725,7 +3725,7 @@ class TestUDTF: def eval(self): yield Row(a=0, b=1.1, c=2), - err = "UDTF_ARROW_DATA_CONVERSION_ERROR" + err = "UDTF_ARROW_TYPE_CONVERSION_ERROR" for ret_type, expected in [ ("x: boolean", err), ("x: tinyint", err), @@ -3761,7 +3761,7 @@ def eval(self): "x: array", ]: with self.subTest(ret_type=ret_type): - with self.assertRaisesRegex(PythonException, "UDTF_ARROW_DATA_CONVERSION_ERROR"): + with self.assertRaisesRegex(PythonException, "UDTF_ARROW_TYPE_CONVERSION_ERROR"): udtf(TestUDTF, returnType=ret_type)().collect() def test_decimal_round(self): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index fe68b8baa6c90..4bae9f6dc48f5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -2342,7 +2342,7 @@ def convert_to_arrow(data: Iterable): def raise_conversion_error(original_exception): raise PySparkRuntimeError( - errorClass="UDTF_ARROW_DATA_CONVERSION_ERROR", + errorClass="UDTF_ARROW_TYPE_CONVERSION_ERROR", messageParameters={ "data": str(data), "schema": return_type.simpleString(), From 23e272b1cbb670a129472f626417259ccfafdade Mon Sep 17 00:00:00 2001 From: Yan Yan Date: Wed, 25 Feb 2026 11:20:49 -0800 Subject: [PATCH 393/400] [SPARK-55694][SQL] Block constraints in CTAS/RTAS at parser level ### What changes were proposed in this pull request? The grammar accepts constraint specifications (PRIMARY KEY, UNIQUE, CHECK, FOREIGN KEY) in CREATE TABLE AS SELECT and REPLACE TABLE AS SELECT, but the execution layer silently drops them. Neither the ANSI SQL standard nor PostgreSQL supports this syntax - the SQL standard makes table element lists and AS subquery clauses mutually exclusive. Block this at the parser level, consistent with existing CTAS/RTAS checks for schema columns and partition column types. ### Why are the changes needed? Explicitly throw exception for an unsupported case for clarity ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? unit test ### Was this patch authored or co-authored using generative AI tooling? Co-Authored-By: Claude Opus 4.6 Closes #54454 from yyanyy/block-ctas-constraints. Authored-by: Yan Yan Signed-off-by: Gengliang Wang (cherry picked from commit 38a75333db72896318177e241b9d0a45d5b99c7b) Signed-off-by: Gengliang Wang --- .../sql/catalyst/parser/AstBuilder.scala | 10 +++++++ .../command/PlanResolutionSuite.scala | 30 +++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 9adc755730f59..6649568a00b20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -5125,6 +5125,11 @@ class AstBuilder extends DataTypeAstBuilder "Partition column types may not be specified in Create Table As Select (CTAS)", ctx) + case Some(_) if constraints.nonEmpty => + operationNotAllowed( + "Constraints may not be specified in a Create Table As Select (CTAS) statement", + ctx) + case Some(query) => CreateTableAsSelect(identifier, partitioning, query, tableSpec, Map.empty, ifNotExists) @@ -5204,6 +5209,11 @@ class AstBuilder extends DataTypeAstBuilder "Partition column types may not be specified in Replace Table As Select (RTAS)", ctx) + case Some(_) if constraints.nonEmpty => + operationNotAllowed( + "Constraints may not be specified in a Replace Table As Select (RTAS) statement", + ctx) + case Some(query) => ReplaceTableAsSelect(identifier, partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 8e5ee1644f9c4..af2efd426b172 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -2968,6 +2968,36 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { ExpectedContext(fragment = sql2, start = 0, stop = 61)) } + test("CTAS statement with constraints") { + Seq( + "CONSTRAINT pk PRIMARY KEY (id)", + "CONSTRAINT uk UNIQUE (id)", + "CONSTRAINT ck CHECK (id > 0)" + ).foreach { constraintDef => + val sql = s"CREATE TABLE ctas1 ($constraintDef) AS SELECT 1 AS id" + assertUnsupported( + sql, + Map("message" -> + "Constraints may not be specified in a Create Table As Select (CTAS) statement"), + ExpectedContext(fragment = sql, start = 0, stop = sql.length - 1)) + } + } + + test("RTAS statement with constraints") { + Seq( + "CONSTRAINT pk PRIMARY KEY (id)", + "CONSTRAINT uk UNIQUE (id)", + "CONSTRAINT ck CHECK (id > 0)" + ).foreach { constraintDef => + val sql = s"REPLACE TABLE rtas1 ($constraintDef) AS SELECT 1 AS id" + assertUnsupported( + sql, + Map("message" -> + "Constraints may not be specified in a Replace Table As Select (RTAS) statement"), + ExpectedContext(fragment = sql, start = 0, stop = sql.length - 1)) + } + } + test("create table - basic") { val query = "CREATE TABLE my_table (id int, name string)" val (desc, allowExisting) = extractTableDesc(query) From e31ed5ddb379a0da49fd2be9042e9036f5266cee Mon Sep 17 00:00:00 2001 From: Yan Yan Date: Wed, 25 Feb 2026 17:04:24 -0800 Subject: [PATCH 394/400] [SPARK-51695][SQL][DOCS][FOLLOW-UP] Clarify NULL handling semantics in UNIQUE constraint Javadoc ### What changes were proposed in this pull request? Document NULLS DISTINCT behavior: NULL values are treated as distinct from each other, so rows with NULLs in unique columns never conflict. Also note that UNIQUE allows nullable columns (unlike PRIMARY KEY) and that NULLS NOT DISTINCT is not currently supported. ### Why are the changes needed? Better javadoc clarity on UNIQUE constraint expectation. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? Co-Authored-By: Claude Opus 4.6 Closes #54357 from yyanyy/unique_definition_clarify. Authored-by: Yan Yan Signed-off-by: Gengliang Wang (cherry picked from commit 53606f21eb1a4dd47be15a2fc353f1dffa23c58d) Signed-off-by: Gengliang Wang --- .../spark/sql/connector/catalog/constraints/Unique.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Unique.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Unique.java index d983ef656297e..d4837932863fd 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Unique.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Unique.java @@ -28,6 +28,15 @@ *

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/dev/test-classes.txt b/dev/test-classes.txt index e69de29bb2d1d..2dc6b290ad4f9 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -0,0 +1,7 @@ +sql/core/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/IntSumUdf.class +sql/core/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/HelloWithPackage.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index e69de29bb2d1d..bd8fc93bc9f0f 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -0,0 +1,17 @@ +core/src/test/resources/TestHelloV2_2.13.jar +core/src/test/resources/TestHelloV3_2.13.jar +core/src/test/resources/TestUDTF.jar +data/artifact-tests/junitLargeJar.jar +data/artifact-tests/smallJar.jar +sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar +sql/connect/client/jvm/src/test/resources/udf2.13.jar +sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar +sql/connect/common/src/test/resources/artifact-tests/smallJar.jar +sql/core/src/test/resources/SPARK-33084.jar +sql/core/src/test/resources/artifact-tests/udf_noA.jar +sql/hive-thriftserver/src/test/resources/TestUDTF.jar +sql/hive/src/test/noclasspath/hive-test-udfs.jar +sql/hive/src/test/resources/SPARK-21101-1.0.jar +sql/hive/src/test/resources/TestUDTF.jar +sql/hive/src/test/resources/data/files/TestSerDe.jar +sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..c89830f127c0c18690cec93ec229c6d3b50f7e95 GIT binary patch literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class new file mode 100644 index 0000000000000000000000000000000000000000..56725764de20539b9c16799d2bff4ee3a4e7de97 GIT binary patch literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..6da55d8b8520dcc03c250a605151cc0d23a45518 GIT binary patch literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar deleted file mode 100644 index 0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q From c0690c763bafabd08e7079d1137fa0a769a05bae Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 11:51:53 +0000 Subject: [PATCH 341/400] Preparing Spark release v4.1.1-rc2 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index d72a6a562432f..0b0b67f74f48b 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.2 +Version: 4.1.1 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index f2afb76fef9ed..1015e44593c57 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index d2d9521f73834..7468310695a6b 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index e5dc64c542f5b..ec57f5ecac8c4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index eb8c7817c8ef0..4e376a5127e07 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index abc36aaa92b0c..899d41eb068ae 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2c4460fac9637..d8277cc5b4b87 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 426ff50535b76..6b4e8e48efc1e 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index cc73ba82b29f1..e5ddfcaa6b5ca 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index f0486ebe8f215..6ebf9ebb24a1c 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 45f640a406784..5542f1901d388 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index bf200867a41d2..9aa3ece76f89f 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index da4366424525a..c67fce0067d2b 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index d89a9f50f4626..fff6c56714f03 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 600a9c34769cf..3cfa9786b04e6 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index c5b85f4027bc2..1243e161e89aa 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index cf092cb94a9e9..12779e9dd2ba4 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index b47c502cadf1f..be6e03719776b 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 2f43af3eff918..97f3c4e8c6d04 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index d7eaf46ea110c..e8bf39701aa18 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index dcf6efff18a9d..5f7ba112b9d3d 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index d65da3de461c2..98d1c0ec4a55c 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 606ab7f01c964..457a9f3bd733e 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 55cd208f70772..0b7b7b32e8dd8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index bd5c50b8ace1c..4b3de4e977445 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.2-SNAPSHOT -SPARK_VERSION_SHORT: 4.1.2 +SPARK_VERSION: 4.1.1 +SPARK_VERSION_SHORT: 4.1.1 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.2"] + 'facetFilters': ["version:4.1.1"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index 2e863dac54a09..e1b54a02e8617 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4448d976179fc..35cb3da6b8c16 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 390bab68a82de..7563f4ae89876 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index c59360214a178..fd914c74ca493 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5b2e4512e5f23..1a3574baf0837 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 80bf3b4053fe6..68e710fb9b659 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index f6eabc8c37c6c..dc757d788121c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 8ee776a66e67d..0a06a60816bd0 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.2.dev0" +__version__: str = "4.1.1" diff --git a/repl/pom.xml b/repl/pom.xml index d83963811474c..2e5d28c3f797e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 9f8980b174e48..388d2d56de408 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f69f81259355f..61d5f97149e1e 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index f46cf6d31392a..04f4dd893ebbb 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index b43610317e1c1..2b3084ff34721 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 5fec0441d49f2..ab95d58e495d5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index c84ae04d3d735..d2f458f8b9fcc 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index f939c328c3cb2..69ebb70b48ede 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 0fddea5d79cbc..268bfb9a37af3 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index 25394b1c6cb88..e38a39859c625 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index d4ee58e87c352..83a42dc2780b6 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 285ea9ae4205c..d2d07a08aa9f8 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 0fec584548f67..8c92db25a7539 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d854c0357657e..4a63c5856adab 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index ea01b79f8e289..1208cf822b1b9 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index ace0daa15d772..08fe9d65d7829 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c712edac1445f..e79faee016fa6 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml From 3e563a4964679db596010d24b73ebda1a5e40d57 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 11:51:53 +0000 Subject: [PATCH 342/400] Revert "Removing test jars and class files" This reverts commit be0aa70245be71c7c0750a7d9a17cf39bf4f5e03. --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 0 -> 3827 bytes core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes data/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes data/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes dev/test-classes.txt | 7 +++++++ dev/test-jars.txt | 17 +++++++++++++++++ .../src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../artifact-tests/smallClassFileDup.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 0 -> 6119 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../artifact-tests/HelloWithPackage.class | Bin 0 -> 635 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 0 -> 1333 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 0 -> 5545 bytes .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 0 -> 35660 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 0 -> 7439 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../test/resources/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 0 -> 19579 bytes 26 files changed, 24 insertions(+) create mode 100644 core/src/test/resources/TestHelloV2_2.13.jar create mode 100644 core/src/test/resources/TestHelloV3_2.13.jar create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100755 data/artifact-tests/junitLargeJar.jar create mode 100755 data/artifact-tests/smallJar.jar create mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar create mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar create mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar create mode 100644 sql/core/src/test/resources/SPARK-33084.jar create mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class create mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class create mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class create mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class create mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar create mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar create mode 100644 sql/hive/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar create mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0c292e7d81ad741dbc11177224b9c30519b7a362 GIT binary patch literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/dev/test-classes.txt b/dev/test-classes.txt index 2dc6b290ad4f9..e69de29bb2d1d 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -1,7 +0,0 @@ -sql/core/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/IntSumUdf.class -sql/core/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/HelloWithPackage.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index bd8fc93bc9f0f..e69de29bb2d1d 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -1,17 +0,0 @@ -core/src/test/resources/TestHelloV2_2.13.jar -core/src/test/resources/TestHelloV3_2.13.jar -core/src/test/resources/TestUDTF.jar -data/artifact-tests/junitLargeJar.jar -data/artifact-tests/smallJar.jar -sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar -sql/connect/client/jvm/src/test/resources/udf2.13.jar -sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar -sql/connect/common/src/test/resources/artifact-tests/smallJar.jar -sql/core/src/test/resources/SPARK-33084.jar -sql/core/src/test/resources/artifact-tests/udf_noA.jar -sql/hive-thriftserver/src/test/resources/TestUDTF.jar -sql/hive/src/test/noclasspath/hive-test-udfs.jar -sql/hive/src/test/resources/SPARK-21101-1.0.jar -sql/hive/src/test/resources/TestUDTF.jar -sql/hive/src/test/resources/data/files/TestSerDe.jar -sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar deleted file mode 100644 index c89830f127c0c18690cec93ec229c6d3b50f7e95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class deleted file mode 100644 index 56725764de20539b9c16799d2bff4ee3a4e7de97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar deleted file mode 100755 index 6da55d8b8520dcc03c250a605151cc0d23a45518..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b GIT binary patch literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q literal 0 HcmV?d00001 From 31ecc1061655848604dba13bf6a707ae39e874ff Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 06:38:43 +0000 Subject: [PATCH 339/400] Preparing development version 4.1.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 0b0b67f74f48b..d72a6a562432f 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.1 +Version: 4.1.2 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 1015e44593c57..f2afb76fef9ed 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 7468310695a6b..d2d9521f73834 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index ec57f5ecac8c4..e5dc64c542f5b 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 4e376a5127e07..eb8c7817c8ef0 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 899d41eb068ae..abc36aaa92b0c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index d8277cc5b4b87..2c4460fac9637 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6b4e8e48efc1e..426ff50535b76 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e5ddfcaa6b5ca..cc73ba82b29f1 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index 6ebf9ebb24a1c..f0486ebe8f215 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 5542f1901d388..45f640a406784 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index 9aa3ece76f89f..bf200867a41d2 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index c67fce0067d2b..da4366424525a 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index fff6c56714f03..d89a9f50f4626 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 3cfa9786b04e6..600a9c34769cf 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index 1243e161e89aa..c5b85f4027bc2 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 12779e9dd2ba4..cf092cb94a9e9 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index be6e03719776b..b47c502cadf1f 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 97f3c4e8c6d04..2f43af3eff918 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index e8bf39701aa18..d7eaf46ea110c 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 5f7ba112b9d3d..dcf6efff18a9d 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 98d1c0ec4a55c..d65da3de461c2 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 457a9f3bd733e..606ab7f01c964 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 0b7b7b32e8dd8..55cd208f70772 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 4b3de4e977445..bd5c50b8ace1c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.1 -SPARK_VERSION_SHORT: 4.1.1 +SPARK_VERSION: 4.1.2-SNAPSHOT +SPARK_VERSION_SHORT: 4.1.2 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.1"] + 'facetFilters': ["version:4.1.2"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index e1b54a02e8617..2e863dac54a09 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 35cb3da6b8c16..4448d976179fc 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 7563f4ae89876..390bab68a82de 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index fd914c74ca493..c59360214a178 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1a3574baf0837..5b2e4512e5f23 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 68e710fb9b659..80bf3b4053fe6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index dc757d788121c..f6eabc8c37c6c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 0a06a60816bd0..8ee776a66e67d 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.1" +__version__: str = "4.1.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 2e5d28c3f797e..d83963811474c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 388d2d56de408..9f8980b174e48 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 61d5f97149e1e..f69f81259355f 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 04f4dd893ebbb..f46cf6d31392a 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 2b3084ff34721..b43610317e1c1 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ab95d58e495d5..5fec0441d49f2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index d2f458f8b9fcc..c84ae04d3d735 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 69ebb70b48ede..f939c328c3cb2 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 268bfb9a37af3..0fddea5d79cbc 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index e38a39859c625..25394b1c6cb88 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 83a42dc2780b6..d4ee58e87c352 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d2d07a08aa9f8..285ea9ae4205c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 8c92db25a7539..0fec584548f67 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 4a63c5856adab..d854c0357657e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 1208cf822b1b9..ea01b79f8e289 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index 08fe9d65d7829..ace0daa15d772 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e79faee016fa6..c712edac1445f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml From be0aa70245be71c7c0750a7d9a17cf39bf4f5e03 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 11:51:01 +0000 Subject: [PATCH 340/400] Removing test jars and class files --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 3827 -> 0 bytes core/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes data/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes data/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes dev/test-classes.txt | 7 ------- dev/test-jars.txt | 17 ----------------- .../src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 5674 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../artifact-tests/smallClassFileDup.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 6119 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../artifact-tests/HelloWithPackage.class | Bin 635 -> 0 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 1333 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 5545 -> 0 bytes .../src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 35660 -> 0 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 7439 -> 0 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../test/resources/data/files/TestSerDe.jar | Bin 3838 -> 0 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 19579 -> 0 bytes 26 files changed, 24 deletions(-) delete mode 100644 core/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 core/src/test/resources/TestHelloV3_2.13.jar delete mode 100644 core/src/test/resources/TestUDTF.jar delete mode 100755 data/artifact-tests/junitLargeJar.jar delete mode 100755 data/artifact-tests/smallJar.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar delete mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar delete mode 100644 sql/core/src/test/resources/SPARK-33084.jar delete mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class delete mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class delete mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class delete mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class delete mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar delete mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar delete mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar delete mode 100644 sql/hive/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar delete mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar deleted file mode 100644 index 0c292e7d81ad741dbc11177224b9c30519b7a362..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar deleted file mode 100644 index 514f2d5d26fd358ad5647e0e75edb8ce77b69e30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/dev/test-classes.txt b/dev/test-classes.txt index e69de29bb2d1d..2dc6b290ad4f9 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -0,0 +1,7 @@ +sql/core/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/IntSumUdf.class +sql/core/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/HelloWithPackage.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index e69de29bb2d1d..bd8fc93bc9f0f 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -0,0 +1,17 @@ +core/src/test/resources/TestHelloV2_2.13.jar +core/src/test/resources/TestHelloV3_2.13.jar +core/src/test/resources/TestUDTF.jar +data/artifact-tests/junitLargeJar.jar +data/artifact-tests/smallJar.jar +sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar +sql/connect/client/jvm/src/test/resources/udf2.13.jar +sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar +sql/connect/common/src/test/resources/artifact-tests/smallJar.jar +sql/core/src/test/resources/SPARK-33084.jar +sql/core/src/test/resources/artifact-tests/udf_noA.jar +sql/hive-thriftserver/src/test/resources/TestUDTF.jar +sql/hive/src/test/noclasspath/hive-test-udfs.jar +sql/hive/src/test/resources/SPARK-21101-1.0.jar +sql/hive/src/test/resources/TestUDTF.jar +sql/hive/src/test/resources/data/files/TestSerDe.jar +sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..c89830f127c0c18690cec93ec229c6d3b50f7e95 GIT binary patch literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class new file mode 100644 index 0000000000000000000000000000000000000000..56725764de20539b9c16799d2bff4ee3a4e7de97 GIT binary patch literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..6da55d8b8520dcc03c250a605151cc0d23a45518 GIT binary patch literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar deleted file mode 100644 index 0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q From f85e4074c2a0ab3f4f90cf55f6937ad85b236dc8 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 06:38:37 +0000 Subject: [PATCH 337/400] Preparing Spark release v4.1.1-rc2 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index d72a6a562432f..0b0b67f74f48b 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.2 +Version: 4.1.1 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index f2afb76fef9ed..1015e44593c57 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index d2d9521f73834..7468310695a6b 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index e5dc64c542f5b..ec57f5ecac8c4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index eb8c7817c8ef0..4e376a5127e07 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index abc36aaa92b0c..899d41eb068ae 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2c4460fac9637..d8277cc5b4b87 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 426ff50535b76..6b4e8e48efc1e 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index cc73ba82b29f1..e5ddfcaa6b5ca 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index f0486ebe8f215..6ebf9ebb24a1c 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 45f640a406784..5542f1901d388 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index bf200867a41d2..9aa3ece76f89f 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index da4366424525a..c67fce0067d2b 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index d89a9f50f4626..fff6c56714f03 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 600a9c34769cf..3cfa9786b04e6 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index c5b85f4027bc2..1243e161e89aa 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index cf092cb94a9e9..12779e9dd2ba4 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index b47c502cadf1f..be6e03719776b 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 2f43af3eff918..97f3c4e8c6d04 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index d7eaf46ea110c..e8bf39701aa18 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index dcf6efff18a9d..5f7ba112b9d3d 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index d65da3de461c2..98d1c0ec4a55c 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 606ab7f01c964..457a9f3bd733e 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 55cd208f70772..0b7b7b32e8dd8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index bd5c50b8ace1c..4b3de4e977445 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.2-SNAPSHOT -SPARK_VERSION_SHORT: 4.1.2 +SPARK_VERSION: 4.1.1 +SPARK_VERSION_SHORT: 4.1.1 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.2"] + 'facetFilters': ["version:4.1.1"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index 2e863dac54a09..e1b54a02e8617 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4448d976179fc..35cb3da6b8c16 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 390bab68a82de..7563f4ae89876 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index c59360214a178..fd914c74ca493 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5b2e4512e5f23..1a3574baf0837 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 80bf3b4053fe6..68e710fb9b659 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index f6eabc8c37c6c..dc757d788121c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 8ee776a66e67d..0a06a60816bd0 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.2.dev0" +__version__: str = "4.1.1" diff --git a/repl/pom.xml b/repl/pom.xml index d83963811474c..2e5d28c3f797e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 9f8980b174e48..388d2d56de408 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f69f81259355f..61d5f97149e1e 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index f46cf6d31392a..04f4dd893ebbb 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index b43610317e1c1..2b3084ff34721 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 5fec0441d49f2..ab95d58e495d5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index c84ae04d3d735..d2f458f8b9fcc 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index f939c328c3cb2..69ebb70b48ede 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 0fddea5d79cbc..268bfb9a37af3 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index 25394b1c6cb88..e38a39859c625 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index d4ee58e87c352..83a42dc2780b6 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 285ea9ae4205c..d2d07a08aa9f8 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 0fec584548f67..8c92db25a7539 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d854c0357657e..4a63c5856adab 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index ea01b79f8e289..1208cf822b1b9 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index ace0daa15d772..08fe9d65d7829 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c712edac1445f..e79faee016fa6 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.2-SNAPSHOT + 4.1.1 ../pom.xml From e7001a07ca353b32b35f4195e9089a114d1b94cf Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 06:38:37 +0000 Subject: [PATCH 338/400] Revert "Removing test jars and class files" This reverts commit 3ba5a6aad586eb43001b0780d6ca716a8bad7637. --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 0 -> 3827 bytes core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes data/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes data/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes dev/test-classes.txt | 7 +++++++ dev/test-jars.txt | 17 +++++++++++++++++ .../src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../artifact-tests/smallClassFileDup.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 0 -> 6119 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../artifact-tests/HelloWithPackage.class | Bin 0 -> 635 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 0 -> 1333 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 0 -> 5545 bytes .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 0 -> 35660 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 0 -> 7439 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../test/resources/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 0 -> 19579 bytes 26 files changed, 24 insertions(+) create mode 100644 core/src/test/resources/TestHelloV2_2.13.jar create mode 100644 core/src/test/resources/TestHelloV3_2.13.jar create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100755 data/artifact-tests/junitLargeJar.jar create mode 100755 data/artifact-tests/smallJar.jar create mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar create mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar create mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar create mode 100644 sql/core/src/test/resources/SPARK-33084.jar create mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class create mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class create mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class create mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class create mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar create mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar create mode 100644 sql/hive/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar create mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0c292e7d81ad741dbc11177224b9c30519b7a362 GIT binary patch literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/dev/test-classes.txt b/dev/test-classes.txt index 2dc6b290ad4f9..e69de29bb2d1d 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -1,7 +0,0 @@ -sql/core/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/IntSumUdf.class -sql/core/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/HelloWithPackage.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index bd8fc93bc9f0f..e69de29bb2d1d 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -1,17 +0,0 @@ -core/src/test/resources/TestHelloV2_2.13.jar -core/src/test/resources/TestHelloV3_2.13.jar -core/src/test/resources/TestUDTF.jar -data/artifact-tests/junitLargeJar.jar -data/artifact-tests/smallJar.jar -sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar -sql/connect/client/jvm/src/test/resources/udf2.13.jar -sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar -sql/connect/common/src/test/resources/artifact-tests/smallJar.jar -sql/core/src/test/resources/SPARK-33084.jar -sql/core/src/test/resources/artifact-tests/udf_noA.jar -sql/hive-thriftserver/src/test/resources/TestUDTF.jar -sql/hive/src/test/noclasspath/hive-test-udfs.jar -sql/hive/src/test/resources/SPARK-21101-1.0.jar -sql/hive/src/test/resources/TestUDTF.jar -sql/hive/src/test/resources/data/files/TestSerDe.jar -sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar deleted file mode 100644 index c89830f127c0c18690cec93ec229c6d3b50f7e95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class deleted file mode 100644 index 56725764de20539b9c16799d2bff4ee3a4e7de97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar deleted file mode 100755 index 6da55d8b8520dcc03c250a605151cc0d23a45518..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b GIT binary patch literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q literal 0 HcmV?d00001 From bfc94ab3104e0c6acbca18b88c5314cb7a13bb9d Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 30 Dec 2025 00:32:17 +0000 Subject: [PATCH 334/400] Preparing development version 4.1.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 0b0b67f74f48b..d72a6a562432f 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.1 +Version: 4.1.2 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 1015e44593c57..f2afb76fef9ed 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 7468310695a6b..d2d9521f73834 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index ec57f5ecac8c4..e5dc64c542f5b 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 4e376a5127e07..eb8c7817c8ef0 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 899d41eb068ae..abc36aaa92b0c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index d8277cc5b4b87..2c4460fac9637 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6b4e8e48efc1e..426ff50535b76 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e5ddfcaa6b5ca..cc73ba82b29f1 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index 6ebf9ebb24a1c..f0486ebe8f215 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 5542f1901d388..45f640a406784 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index 9aa3ece76f89f..bf200867a41d2 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index c67fce0067d2b..da4366424525a 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index fff6c56714f03..d89a9f50f4626 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 3cfa9786b04e6..600a9c34769cf 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index 1243e161e89aa..c5b85f4027bc2 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 12779e9dd2ba4..cf092cb94a9e9 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index be6e03719776b..b47c502cadf1f 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 97f3c4e8c6d04..2f43af3eff918 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index e8bf39701aa18..d7eaf46ea110c 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 5f7ba112b9d3d..dcf6efff18a9d 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 98d1c0ec4a55c..d65da3de461c2 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 457a9f3bd733e..606ab7f01c964 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 0b7b7b32e8dd8..55cd208f70772 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 4b3de4e977445..bd5c50b8ace1c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.1 -SPARK_VERSION_SHORT: 4.1.1 +SPARK_VERSION: 4.1.2-SNAPSHOT +SPARK_VERSION_SHORT: 4.1.2 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.1"] + 'facetFilters': ["version:4.1.2"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index e1b54a02e8617..2e863dac54a09 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 35cb3da6b8c16..4448d976179fc 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 7563f4ae89876..390bab68a82de 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index fd914c74ca493..c59360214a178 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1a3574baf0837..5b2e4512e5f23 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 68e710fb9b659..80bf3b4053fe6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index dc757d788121c..f6eabc8c37c6c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 0a06a60816bd0..8ee776a66e67d 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.1" +__version__: str = "4.1.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 2e5d28c3f797e..d83963811474c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 388d2d56de408..9f8980b174e48 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 61d5f97149e1e..f69f81259355f 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 04f4dd893ebbb..f46cf6d31392a 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 2b3084ff34721..b43610317e1c1 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ab95d58e495d5..5fec0441d49f2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index d2f458f8b9fcc..c84ae04d3d735 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 69ebb70b48ede..f939c328c3cb2 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 268bfb9a37af3..0fddea5d79cbc 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index e38a39859c625..25394b1c6cb88 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 83a42dc2780b6..d4ee58e87c352 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d2d07a08aa9f8..285ea9ae4205c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 8c92db25a7539..0fec584548f67 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 4a63c5856adab..d854c0357657e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 1208cf822b1b9..ea01b79f8e289 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index 08fe9d65d7829..ace0daa15d772 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e79faee016fa6..c712edac1445f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1 + 4.1.2-SNAPSHOT ../pom.xml From 6d1113b20f4cba96a5f125ef11071e5d8684ca00 Mon Sep 17 00:00:00 2001 From: xihuan_mstr Date: Fri, 2 Jan 2026 09:02:46 +0900 Subject: [PATCH 335/400] [SPARK-54753][SQL][4.1] Fix memory leak of ArtifactManager ### What changes were proposed in this pull request? As stated in https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/lang/ref/Cleaner.html **The cleaning action could be a lambda but all too easily will capture the object reference, by referring to fields of the object being cleaned, preventing the object from becoming phantom reachable. Using a static nested class, as above, will avoid accidentally retaining the object reference.** For more details, and the test and analysis are in https://issues.apache.org/jira/browse/SPARK-54753 image After running with Spark 4.0.1, the ArtififactManager is leaked, its referenced SessionState/SparkSession is as well leaked. ### Why are the changes needed? use a separate class to ref the cleanup state ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? with test program in https://issues.apache.org/jira/browse/SPARK-54753, and use Visual VM to monitor the memory usage ### Was this patch authored or co-authored using generative AI tooling? No **This PR backports the fix in https://github.com/apache/spark/pull/53591 to branch 4.1** cc dongjoon-hyun pranavdev022 hvanhovell vicennial HyukjinKwon Closes #53654 from scottme/SPARK-54753-4.1. Authored-by: xihuan_mstr Signed-off-by: Hyukjin Kwon --- .../client/CheckConnectJvmClientCompatibility.scala | 2 ++ .../org/apache/spark/sql/artifact/ArtifactManager.scala | 9 +++++++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 92adc8eb93461..1a6fd7926dafe 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -234,6 +234,8 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.artifact.ArtifactManager$"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.artifact.ArtifactManager$SparkContextResourceType$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.artifact.ArtifactManager$StateCleanupRunner"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.artifact.RefCountedCacheId"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index 346cdb832c3ff..e11f0f99bf2f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -422,8 +422,7 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging artifactPath) // Ensure that no reference to `this` is captured/help by the cleanup lambda private def getCleanable: Cleaner.Cleanable = cleaner.register( - this, - () => ArtifactManager.cleanUpGlobalResources(cleanUpStateForGlobalResources) + this, new StateCleanupRunner(cleanUpStateForGlobalResources) ) private var cleanable = getCleanable @@ -529,6 +528,12 @@ object ArtifactManager extends Logging { val JAR, FILE, ARCHIVE = Value } + private class StateCleanupRunner(cleanupState: ArtifactStateForCleanup) extends Runnable { + override def run(): Unit = { + ArtifactManager.cleanUpGlobalResources(cleanupState) + } + } + // Shared cleaner instance private val cleaner: Cleaner = Cleaner.create() From 3ba5a6aad586eb43001b0780d6ca716a8bad7637 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Jan 2026 06:38:06 +0000 Subject: [PATCH 336/400] Removing test jars and class files --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 3827 -> 0 bytes core/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes data/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes data/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes dev/test-classes.txt | 7 ------- dev/test-jars.txt | 17 ----------------- .../src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 5674 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../artifact-tests/smallClassFileDup.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 6119 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../artifact-tests/HelloWithPackage.class | Bin 635 -> 0 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 1333 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 5545 -> 0 bytes .../src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 35660 -> 0 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 7439 -> 0 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../test/resources/data/files/TestSerDe.jar | Bin 3838 -> 0 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 19579 -> 0 bytes 26 files changed, 24 deletions(-) delete mode 100644 core/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 core/src/test/resources/TestHelloV3_2.13.jar delete mode 100644 core/src/test/resources/TestUDTF.jar delete mode 100755 data/artifact-tests/junitLargeJar.jar delete mode 100755 data/artifact-tests/smallJar.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar delete mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar delete mode 100644 sql/core/src/test/resources/SPARK-33084.jar delete mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class delete mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class delete mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class delete mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class delete mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar delete mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar delete mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar delete mode 100644 sql/hive/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar delete mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar deleted file mode 100644 index 0c292e7d81ad741dbc11177224b9c30519b7a362..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar deleted file mode 100644 index 514f2d5d26fd358ad5647e0e75edb8ce77b69e30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/dev/test-classes.txt b/dev/test-classes.txt index e69de29bb2d1d..2dc6b290ad4f9 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -0,0 +1,7 @@ +sql/core/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/IntSumUdf.class +sql/core/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/HelloWithPackage.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index e69de29bb2d1d..bd8fc93bc9f0f 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -0,0 +1,17 @@ +core/src/test/resources/TestHelloV2_2.13.jar +core/src/test/resources/TestHelloV3_2.13.jar +core/src/test/resources/TestUDTF.jar +data/artifact-tests/junitLargeJar.jar +data/artifact-tests/smallJar.jar +sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar +sql/connect/client/jvm/src/test/resources/udf2.13.jar +sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar +sql/connect/common/src/test/resources/artifact-tests/smallJar.jar +sql/core/src/test/resources/SPARK-33084.jar +sql/core/src/test/resources/artifact-tests/udf_noA.jar +sql/hive-thriftserver/src/test/resources/TestUDTF.jar +sql/hive/src/test/noclasspath/hive-test-udfs.jar +sql/hive/src/test/resources/SPARK-21101-1.0.jar +sql/hive/src/test/resources/TestUDTF.jar +sql/hive/src/test/resources/data/files/TestSerDe.jar +sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..c89830f127c0c18690cec93ec229c6d3b50f7e95 GIT binary patch literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class new file mode 100644 index 0000000000000000000000000000000000000000..56725764de20539b9c16799d2bff4ee3a4e7de97 GIT binary patch literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..6da55d8b8520dcc03c250a605151cc0d23a45518 GIT binary patch literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar deleted file mode 100644 index 0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q From 639216180c74b4ac6c660c19a5fb83197c57f5e1 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 30 Dec 2025 00:32:11 +0000 Subject: [PATCH 332/400] Preparing Spark release v4.1.1-rc1 --- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 2 +- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 49 files changed, 49 insertions(+), 49 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 0bd7673d60185..1015e44593c57 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f26b72b08efec..7468310695a6b 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5a4dc11f51fcc..ec57f5ecac8c4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 794b8e9c6872e..4e376a5127e07 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 21507aa76efaf..899d41eb068ae 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8e51d311be5f4..d8277cc5b4b87 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index cc36b54df8519..6b4e8e48efc1e 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 130153d6127a7..e5ddfcaa6b5ca 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index 114086da25cb3..6ebf9ebb24a1c 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 981d484451865..5542f1901d388 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index 4ff99df6350fc..9aa3ece76f89f 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index f6aa500d5f007..c67fce0067d2b 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index dc01425a617fe..fff6c56714f03 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 4f1eb8302aebc..3cfa9786b04e6 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index f6b7dbcdc3c2a..1243e161e89aa 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 758ea445746df..12779e9dd2ba4 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index cee71d4ccf453..be6e03719776b 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 57694e27cecaa..97f3c4e8c6d04 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index 91f278500d78f..e8bf39701aa18 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 356934c9eb929..5f7ba112b9d3d 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 7abbc76899879..98d1c0ec4a55c 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 545017f1d77d1..457a9f3bd733e 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2719881e25cf4..0b7b7b32e8dd8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index bda945e03c203..4b3de4e977445 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,7 +19,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.1-SNAPSHOT +SPARK_VERSION: 4.1.1 SPARK_VERSION_SHORT: 4.1.1 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" diff --git a/examples/pom.xml b/examples/pom.xml index 38daf00501ce6..e1b54a02e8617 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cc4327ccff090..35cb3da6b8c16 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 66b8cb104e07f..7563f4ae89876 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index aa8812cd9a1f6..fd914c74ca493 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 34df7e4b5be1d..1a3574baf0837 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9061edd455ebf..68e710fb9b659 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 1e61d457e20e9..dc757d788121c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 3080cb455888b..0a06a60816bd0 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.1.dev0" +__version__: str = "4.1.1" diff --git a/repl/pom.xml b/repl/pom.xml index 74199753f2c5e..2e5d28c3f797e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 3e58d34fc1857..388d2d56de408 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 6051496bf7fe6..61d5f97149e1e 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index f1a8c05034f60..04f4dd893ebbb 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 7d33b1e5f107e..2b3084ff34721 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index d7b0d7e0554bb..ab95d58e495d5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index 6dd373071f6c2..d2f458f8b9fcc 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 40621d51d7dcb..69ebb70b48ede 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index ecd735cb81ca5..268bfb9a37af3 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index 24d74b505521f..e38a39859c625 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 79846203864e1..83a42dc2780b6 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 270d23061f391..d2d07a08aa9f8 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a1966cace77c8..8c92db25a7539 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 55086c6b5de8d..4a63c5856adab 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 94de01aede9bc..1208cf822b1b9 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index ffac010cad1cc..08fe9d65d7829 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 934ff72708094..e79faee016fa6 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.1 ../pom.xml From 4a628d8c700319da77a87c2f6860bb3261e53960 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 30 Dec 2025 00:32:11 +0000 Subject: [PATCH 333/400] Revert "Removing test jars and class files" This reverts commit 554af03a281d211a251da701bfbd53577697f117. --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 0 -> 3827 bytes core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes data/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes data/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes dev/test-classes.txt | 7 +++++++ dev/test-jars.txt | 17 +++++++++++++++++ .../src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../artifact-tests/smallClassFileDup.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 0 -> 6119 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../artifact-tests/HelloWithPackage.class | Bin 0 -> 635 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 0 -> 1333 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 0 -> 5545 bytes .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 0 -> 35660 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 0 -> 7439 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../test/resources/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 0 -> 19579 bytes 26 files changed, 24 insertions(+) create mode 100644 core/src/test/resources/TestHelloV2_2.13.jar create mode 100644 core/src/test/resources/TestHelloV3_2.13.jar create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100755 data/artifact-tests/junitLargeJar.jar create mode 100755 data/artifact-tests/smallJar.jar create mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar create mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar create mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar create mode 100644 sql/core/src/test/resources/SPARK-33084.jar create mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class create mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class create mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class create mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class create mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar create mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar create mode 100644 sql/hive/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar create mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0c292e7d81ad741dbc11177224b9c30519b7a362 GIT binary patch literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/dev/test-classes.txt b/dev/test-classes.txt index 2dc6b290ad4f9..e69de29bb2d1d 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -1,7 +0,0 @@ -sql/core/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/IntSumUdf.class -sql/core/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/HelloWithPackage.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index bd8fc93bc9f0f..e69de29bb2d1d 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -1,17 +0,0 @@ -core/src/test/resources/TestHelloV2_2.13.jar -core/src/test/resources/TestHelloV3_2.13.jar -core/src/test/resources/TestUDTF.jar -data/artifact-tests/junitLargeJar.jar -data/artifact-tests/smallJar.jar -sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar -sql/connect/client/jvm/src/test/resources/udf2.13.jar -sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar -sql/connect/common/src/test/resources/artifact-tests/smallJar.jar -sql/core/src/test/resources/SPARK-33084.jar -sql/core/src/test/resources/artifact-tests/udf_noA.jar -sql/hive-thriftserver/src/test/resources/TestUDTF.jar -sql/hive/src/test/noclasspath/hive-test-udfs.jar -sql/hive/src/test/resources/SPARK-21101-1.0.jar -sql/hive/src/test/resources/TestUDTF.jar -sql/hive/src/test/resources/data/files/TestSerDe.jar -sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar deleted file mode 100644 index c89830f127c0c18690cec93ec229c6d3b50f7e95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class deleted file mode 100644 index 56725764de20539b9c16799d2bff4ee3a4e7de97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar deleted file mode 100755 index 6da55d8b8520dcc03c250a605151cc0d23a45518..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b GIT binary patch literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q literal 0 HcmV?d00001 From c823c4a0f0e0cf8dc59280a00e699199739625f6 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Thu, 11 Dec 2025 14:01:00 +0000 Subject: [PATCH 293/400] Preparing development version 4.1.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 4393175430265..0b0b67f74f48b 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.0 +Version: 4.1.1 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 743f74753b1f8..0bd7673d60185 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 1afbf81679906..f26b72b08efec 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 167582598a920..5a4dc11f51fcc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 8cede6cbf288e..794b8e9c6872e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index d17a0876fe420..21507aa76efaf 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 161802d43ca89..8e51d311be5f4 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 5e851791a6975..cc36b54df8519 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 869931e403349..130153d6127a7 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index dbac5e4552df5..114086da25cb3 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 1805f8462c278..981d484451865 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index adecb4b4185d3..4ff99df6350fc 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index bad1f7529c054..f6aa500d5f007 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index b5af1a093be36..dc01425a617fe 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 0022adff1a208..4f1eb8302aebc 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index b482aeb77a96d..f6b7dbcdc3c2a 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 5da0ee078e26e..758ea445746df 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index 2479590341822..cee71d4ccf453 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 9acc9b65d7653..57694e27cecaa 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index 54d5747e78aef..91f278500d78f 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 9d486a868c3de..356934c9eb929 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index a78b722eb6bf0..7abbc76899879 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 7ee7e096d38c8..545017f1d77d1 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2281f3e197f88..2719881e25cf4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index dc3890e4522ee..bda945e03c203 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.0 -SPARK_VERSION_SHORT: 4.1.0 +SPARK_VERSION: 4.1.1-SNAPSHOT +SPARK_VERSION_SHORT: 4.1.1 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.0"] + 'facetFilters': ["version:4.1.1"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index cd17565a1d98b..38daf00501ce6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 602817ca792e4..cc4327ccff090 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index fc593ff2c6196..66b8cb104e07f 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e49a0678d7182..aa8812cd9a1f6 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 6695e77228935..34df7e4b5be1d 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index b31c2b7fb3a24..9061edd455ebf 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 1824a28614bd3..1e61d457e20e9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 9b0e7c1f5ce40..3080cb455888b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.0" +__version__: str = "4.1.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 53bc41d397d60..74199753f2c5e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index c5f2749ea9fb0..3e58d34fc1857 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 3b471a8ff4133..6051496bf7fe6 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 6d0d0305ef710..f1a8c05034f60 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index e7997cf61a457..7d33b1e5f107e 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 8eef00489669d..d7b0d7e0554bb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index 07d7096ea255f..6dd373071f6c2 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 6532afa46710a..40621d51d7dcb 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index 3b74dca497387..ecd735cb81ca5 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index ab19897f64285..24d74b505521f 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index febf7a2beddf4..79846203864e1 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e3aa8f1f3bb4c..270d23061f391 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 8c4087372913a..a1966cace77c8 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index e51872a36ca5b..55086c6b5de8d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 6e4a4d64f7930..94de01aede9bc 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index 9602aa01e7c4d..ffac010cad1cc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index cfc6a7150d858..934ff72708094 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.0 + 4.1.1-SNAPSHOT ../pom.xml From b29c8223639bb795f57cf0a9a531750a3ed507f1 Mon Sep 17 00:00:00 2001 From: Yi Wu Date: Mon, 15 Dec 2025 10:47:14 +0800 Subject: [PATCH 294/400] [SPARK-54695][TEST][CORE] StandaloneDynamicAllocationSuite.syncExecutors should ensure executors have fully setup ### What changes were proposed in this pull request? This PR fixes `StandaloneDynamicAllocationSuite.syncExecutors()` to ensure executors have fully setup before it returns. ### Why are the changes needed? After SPARK-29287, executor gets fully setup in two steps: 1. `RegisterExecutor`, 2. `LaunchedExecutor`. And the executor is only ready for use (`freeCores` > 0) when the driver has processed the `LaunchedExecutor`. `StandaloneDynamicAllocationSuite.syncExecutors()` was unintentionally broken by SPARK-29287, leading to unexpected access failures upon executor's `freeCores`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Covered by existing tests (improve the statabilty) ### Was this patch authored or co-authored using generative AI tooling? No Closes #53457 from Ngone51/fix-syncExecutors. Authored-by: Yi Wu Signed-off-by: Wenchen Fan (cherry picked from commit 45a2604df52ad97758c3081f0cf6e2b13b668983) Signed-off-by: Wenchen Fan --- .../apache/spark/deploy/StandaloneDynamicAllocationSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 5ecc551c16b8c..487a90f157a9b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -632,6 +632,9 @@ class StandaloneDynamicAllocationSuite Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) backend.driverEndpoint.askSync[Boolean](message) backend.driverEndpoint.send(LaunchedExecutor(id)) + eventually(timeout(10.seconds), interval(100.millis)) { + assert(backend.getExecutorAvailableCpus(id).exists(_ > 0)) + } } } From f8e390b8e7faf5d6f253012250e5fa8b402c1720 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 15 Dec 2025 10:51:55 +0800 Subject: [PATCH 295/400] [SPARK-7008][SDP][ML] Make `org.apache.spark.sql.pipelines` internal package and make `EstimatorUtils` private ### What changes were proposed in this pull request? Make org.apache.spark.sql.pipelines internal package and make EstimatorUtils private ### Why are the changes needed? they should not be exposed to end users ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53444 from zhengruifeng/mk_EstimatorUtils_private. Authored-by: Ruifeng Zheng Signed-off-by: Wenchen Fan (cherry picked from commit 348164964c1d070d257c717aac914c460630c58a) Signed-off-by: Wenchen Fan --- mllib/src/main/scala/org/apache/spark/ml/Estimator.scala | 2 +- project/SparkBuild.scala | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index 0e1f64cc7b630..84ced24414379 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -106,7 +106,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage { } -object EstimatorUtils { +private[spark] object EstimatorUtils { // This warningMessagesBuffer is for collecting warning messages during `estimator.fit` // execution in Spark Connect server. private[spark] val warningMessagesBuffer = new java.lang.ThreadLocal[ArrayBuffer[String]]() { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7a4c3f0828d03..fec6a3346f7d2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -366,6 +366,7 @@ object SparkBuild extends PomBuild { "org.apache.spark.kafka010", "org.apache.spark.network", "org.apache.spark.sql.avro", + "org.apache.spark.sql.pipelines", "org.apache.spark.sql.scripting", "org.apache.spark.types.variant", "org.apache.spark.ui.flamegraph", @@ -1530,6 +1531,7 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/classic/"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/internal"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/pipelines"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/scripting"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/ml"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive"))) From 6327add8b82285856828759976cfe0a24958fd15 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 15 Dec 2025 11:01:18 +0800 Subject: [PATCH 296/400] [SPARK-54683][SQL] Unify geo and time types blocking This PR aims to refactor the code that blocks time and geo types. code unification no existing tests no Closes #53438 from cloud-fan/block. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 4a18179d6abcd17e07ab4fee8a22b12f3d90ef7f) Signed-off-by: Wenchen Fan --- .../sql/catalyst/CatalystTypeConverters.scala | 10 +++------ .../catalyst/DeserializerBuildHelper.scala | 2 ++ .../sql/catalyst/SerializerBuildHelper.scala | 2 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 1 + .../spark/sql/catalyst/expressions/Cast.scala | 21 ++----------------- .../plans/logical/v2AlterTableCommands.scala | 2 ++ .../spark/sql/catalyst/util/TypeUtils.scala | 13 ++++++++++++ .../execution/SparkConnectPlanExecution.scala | 16 +++----------- .../datasources/DataSourceUtils.scala | 6 ++---- 9 files changed, 30 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index a90720ac5108e..cdbd2d49e8b74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -25,11 +25,12 @@ import java.time.{Duration, Instant, LocalDate, LocalDateTime, LocalTime, Period import java.util.{Map => JavaMap} import javax.annotation.Nullable +import scala.language.existentials + import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DayTimeIntervalType._ @@ -60,6 +61,7 @@ object CatalystTypeConverters { } private def getConverterForType(dataType: DataType): CatalystTypeConverter[Any, Any, Any] = { + TypeUtils.failUnsupportedDataType(dataType, SQLConf.get) val converter = dataType match { case udt: UserDefinedType[_] => UDTConverter(udt) case arrayType: ArrayType => ArrayConverter(arrayType.elementType) @@ -68,18 +70,12 @@ object CatalystTypeConverters { case CharType(length) => new CharConverter(length) case VarcharType(length) => new VarcharConverter(length) case _: StringType => StringConverter - case _ @ (_: GeographyType | _: GeometryType) if !SQLConf.get.geospatialEnabled => - throw new org.apache.spark.sql.AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", - messageParameters = scala.collection.immutable.Map.empty) case g: GeographyType => new GeographyConverter(g) case g: GeometryType => new GeometryConverter(g) case DateType if SQLConf.get.datetimeJava8ApiEnabled => LocalDateConverter case DateType => DateConverter - case _: TimeType if !SQLConf.get.isTimeTypeEnabled => - QueryCompilationErrors.unsupportedTimeTypeError() case _: TimeType => TimeConverter case TimestampType if SQLConf.get.datetimeJava8ApiEnabled => InstantConverter case TimestampType => TimestampConverter diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index 85a5cf4f6b26a..080794643fa0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -345,6 +345,8 @@ object DeserializerBuildHelper { createDeserializerForInstant(path) case LocalDateTimeEncoder => createDeserializerForLocalDateTime(path) + case LocalTimeEncoder if !SQLConf.get.isTimeTypeEnabled => + throw org.apache.spark.sql.errors.QueryCompilationErrors.unsupportedTimeTypeError() case LocalTimeEncoder => createDeserializerForLocalTime(path) case UDTEncoder(udt, udtClass) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index 32fb859745d89..b8b2406a58130 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -367,6 +367,8 @@ object SerializerBuildHelper { case TimestampEncoder(false) => createSerializerForSqlTimestamp(input) case InstantEncoder(false) => createSerializerForJavaInstant(input) case LocalDateTimeEncoder => createSerializerForLocalDateTime(input) + case LocalTimeEncoder if !SQLConf.get.isTimeTypeEnabled => + throw org.apache.spark.sql.errors.QueryCompilationErrors.unsupportedTimeTypeError() case LocalTimeEncoder => createSerializerForLocalTime(input) case UDTEncoder(udt, udtClass) => createSerializerForUserDefinedType(input, udt, udtClass) case OptionEncoder(valueEnc) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 04451162276fb..3b8a363e704a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -712,6 +712,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } create.tableSchema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) + TypeUtils.failUnsupportedDataType(create.tableSchema, SQLConf.get) SchemaUtils.checkIndeterminateCollationInSchema(create.tableSchema) case write: V2WriteCommand if write.resolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 03b6a452fe7da..00b0a83f6d533 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.catalyst.util.IntervalUtils.{dayTimeIntervalToByte, dayTimeIntervalToDecimal, dayTimeIntervalToInt, dayTimeIntervalToLong, dayTimeIntervalToShort, yearMonthIntervalToByte, yearMonthIntervalToInt, yearMonthIntervalToShort} -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase, QueryExecutionErrors} +import org.apache.spark.sql.errors.{QueryErrorsBase, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{GeographyVal, UTF8String, VariantVal} @@ -90,12 +90,6 @@ object Cast extends QueryErrorsBase { * - String <=> Binary */ def canAnsiCast(from: DataType, to: DataType): Boolean = (from, to) match { - case (fromType, toType) if !SQLConf.get.geospatialEnabled && - (isGeoSpatialType(fromType) || isGeoSpatialType(toType)) => - throw new org.apache.spark.sql.AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", - messageParameters = scala.collection.immutable.Map.empty) - case (fromType, toType) if fromType == toType => true case (NullType, _) => true @@ -224,12 +218,6 @@ object Cast extends QueryErrorsBase { * Returns true iff we can cast `from` type to `to` type. */ def canCast(from: DataType, to: DataType): Boolean = (from, to) match { - case (fromType, toType) if !SQLConf.get.geospatialEnabled && - (isGeoSpatialType(fromType) || isGeoSpatialType(toType)) => - throw new org.apache.spark.sql.AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", - messageParameters = scala.collection.immutable.Map.empty) - case (fromType, toType) if fromType == toType => true case (NullType, _) => true @@ -602,12 +590,7 @@ case class Cast( } override def checkInputDataTypes(): TypeCheckResult = { - dataType match { - // If the cast is to a TIME type, first check if TIME type is enabled. - case _: TimeType if !SQLConf.get.isTimeTypeEnabled => - throw QueryCompilationErrors.unsupportedTimeTypeError() - case _ => - } + TypeUtils.failUnsupportedDataType(dataType, SQLConf.get) val canCast = evalMode match { case EvalMode.LEGACY => Cast.canCast(child.dataType, dataType) case EvalMode.ANSI => Cast.canAnsiCast(child.dataType, dataType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index 4ec8baf351cb8..843ce22061d8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -112,6 +112,7 @@ case class AddColumns( columnsToAdd: Seq[QualifiedColType]) extends AlterTableCommand { columnsToAdd.foreach { c => TypeUtils.failWithIntervalType(c.dataType) + TypeUtils.failUnsupportedDataType(c.dataType, conf) } override lazy val resolved: Boolean = table.resolved && columnsToAdd.forall(_.resolved) @@ -144,6 +145,7 @@ case class ReplaceColumns( columnsToAdd: Seq[QualifiedColType]) extends AlterTableCommand { columnsToAdd.foreach { c => TypeUtils.failWithIntervalType(c.dataType) + TypeUtils.failUnsupportedDataType(c.dataType, conf) } override lazy val resolved: Boolean = table.resolved && columnsToAdd.forall(_.resolved) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala index 9f89f068b7568..9c5df04f9569a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala @@ -20,8 +20,10 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.sql.catalyst.analysis.{AnalysisErrorAt, TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.{Expression, RowOrdering} +import org.apache.spark.sql.catalyst.expressions.st.STExpressionUtils.isGeoSpatialType import org.apache.spark.sql.catalyst.types.{PhysicalDataType, PhysicalNumericType} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -136,4 +138,15 @@ object TypeUtils extends QueryErrorsBase { } if (dataType.existsRecursively(isInterval)) f } + + def failUnsupportedDataType(dataType: DataType, conf: SQLConf): Unit = { + if (!conf.isTimeTypeEnabled && dataType.existsRecursively(_.isInstanceOf[TimeType])) { + throw QueryCompilationErrors.unsupportedTimeTypeError() + } + if (!conf.geospatialEnabled && dataType.existsRecursively(isGeoSpatialType)) { + throw new org.apache.spark.sql.AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", + messageParameters = scala.collection.immutable.Map.empty) + } + } } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index e3c392cd2a2c7..4b12c96e977e6 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -29,7 +29,7 @@ import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.st.STExpressionUtils +import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.classic.{DataFrame, Dataset} import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto @@ -37,11 +37,10 @@ import org.apache.spark.sql.connect.config.Connect.{CONNECT_GRPC_ARROW_MAX_BATCH import org.apache.spark.sql.connect.planner.{InvalidInputErrors, SparkConnectPlanner} import org.apache.spark.sql.connect.service.ExecuteHolder import org.apache.spark.sql.connect.utils.MetricGenerator -import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.{DoNotCleanup, LocalTableScanExec, QueryExecution, RemoveShuffleFiles, SkipMigration, SQLExecution} import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, StructType, TimeType} +import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ThreadUtils /** @@ -128,16 +127,7 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) val sessionId = executePlan.sessionHolder.sessionId val spark = dataframe.sparkSession val schema = dataframe.schema - val geospatialEnabled = spark.sessionState.conf.geospatialEnabled - if (!geospatialEnabled && schema.existsRecursively(STExpressionUtils.isGeoSpatialType)) { - throw new org.apache.spark.sql.AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.GEOSPATIAL_DISABLED", - messageParameters = scala.collection.immutable.Map.empty) - } - val timeTypeEnabled = spark.sessionState.conf.isTimeTypeEnabled - if (!timeTypeEnabled && schema.existsRecursively(_.isInstanceOf[TimeType])) { - throw QueryCompilationErrors.unsupportedTimeTypeError() - } + TypeUtils.failUnsupportedDataType(schema, spark.sessionState.conf) val maxRecordsPerBatch = spark.sessionState.conf.arrowMaxRecordsPerBatch val timeZoneId = spark.sessionState.conf.sessionLocalTimeZone val largeVarTypes = spark.sessionState.conf.arrowUseLargeVarTypes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala index fb5b605bab01e..3fd82573f001a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkException, SparkUpgradeException} import org.apache.spark.sql.{sources, SPARK_LEGACY_DATETIME_METADATA_KEY, SPARK_LEGACY_INT96_METADATA_KEY, SPARK_TIMEZONE_METADATA_KEY, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, ExpressionSet, PredicateHelper} -import org.apache.spark.sql.catalyst.util.RebaseDateTime +import org.apache.spark.sql.catalyst.util.{RebaseDateTime, TypeUtils} import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -93,9 +93,7 @@ object DataSourceUtils extends PredicateHelper { * in a driver side. */ def verifySchema(format: FileFormat, schema: StructType, readOnly: Boolean = false): Unit = { - if (!SQLConf.get.isTimeTypeEnabled && schema.existsRecursively(_.isInstanceOf[TimeType])) { - throw QueryCompilationErrors.unsupportedTimeTypeError() - } + TypeUtils.failUnsupportedDataType(schema, SQLConf.get) schema.foreach { field => val supported = if (readOnly) { format.supportReadDataType(field.dataType) From 13b89bad552e0638eae23268b308508f7221627e Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 15 Dec 2025 11:09:27 +0800 Subject: [PATCH 297/400] [SPARK-54692][DOCS] Add python_worker_logs tvf doc to API reference ### What changes were proposed in this pull request? Adds `python_worker_logs` tvf doc to API reference. ### Why are the changes needed? The doc is missing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53451 from ueshin/issues/SPARK-54692/doc. Authored-by: Takuya Ueshin Signed-off-by: Ruifeng Zheng (cherry picked from commit 48c323aa6e26e879a9b71b60d19aacb853751712) Signed-off-by: Ruifeng Zheng --- python/docs/source/reference/pyspark.sql/functions.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index f6ce1a3f2d065..5fed5c8d6719d 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -711,6 +711,7 @@ Table-Valued Functions TableValuedFunction.json_tuple TableValuedFunction.posexplode TableValuedFunction.posexplode_outer + TableValuedFunction.python_worker_logs TableValuedFunction.range TableValuedFunction.sql_keywords TableValuedFunction.stack From 4d3cd058f065a66919d346c398f81692b4021dcb Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 15 Dec 2025 13:51:10 +0800 Subject: [PATCH 298/400] [SPARK-53991][SQL][FOLLOWUP] Enforce KLL_SKETCH_AGG_GET_RANK/QUANTILE arguments are foldable ### What changes were proposed in this pull request? This PR adds a restriction that the rank/quantile arguments to `kll_sketch_get_quantile_*` and `kll_sketch_get_rank_*` functions must be foldable (compile-time constants). **Changes:** 1. Added `checkInputDataTypes()` validation to `KllSketchGetQuantileBase` and `KllSketchGetRankBase` that returns a `NON_FOLDABLE_INPUT` error if the rank/quantile argument is not foldable. 2. Fixed a bug in `nullSafeEval` where `right.eval()` was being called instead of using the already-evaluated `rightInput` parameter. 3. Added negative test cases in `kllquantiles.sql` to verify the new foldability restriction for: - Non-foldable scalar rank argument to `kll_sketch_get_quantile_bigint` - Non-foldable array rank argument to `kll_sketch_get_quantile_bigint` - Non-foldable scalar quantile argument to `kll_sketch_get_rank_bigint` - Non-foldable array quantile argument to `kll_sketch_get_rank_bigint` ### Why are the changes needed? The foldability restriction is reasonable to have, since this is the intended usage of the functions, and is consistent with other existing functions in Spark. Additionally, fixing the `right.eval()` bug ensures we use the already-evaluated value passed to `nullSafeEval`, avoiding redundant evaluation. ### Does this PR introduce _any_ user-facing change? Yes. Users will now receive an error if they pass a non-constant expression (e.g., a column reference) as the rank argument to `kll_sketch_get_quantile_*` or the quantile argument to `kll_sketch_get_rank_*`. For example: -- This will now fail with NON_FOLDABLE_INPUT error SELECT kll_sketch_get_quantile_bigint(sketch, col / 10.0) FROM table; -- This is allowed (constant expression) SELECT kll_sketch_get_quantile_bigint(sketch, 0.5) FROM table; SELECT kll_sketch_get_quantile_bigint(sketch, array(0.25, 0.5, 0.75)) FROM table;### How was this patch tested? Added negative test cases in `kllquantiles.sql` that exercise the new foldability restriction for both scalar and array inputs. The tests verify that appropriate error messages are returned for non-foldable arguments. ### Was this patch authored or co-authored using generative AI tooling? Yes, with assistance from `claude-4.5-opus-high` with manual review and adjustment. Closes #53463 from dtenedor/kll-get-quantile-rank-check-foldable. Lead-authored-by: Daniel Tenedorio Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 392666f167ba47a9a1e31d9a39cb3fb03369caa5) Signed-off-by: Wenchen Fan --- .../catalyst/expressions/kllExpressions.scala | 36 +++++- .../analyzer-results/kllquantiles.sql.out | 112 ++++++++++++++++ .../sql-tests/inputs/kllquantiles.sql | 35 +++++ .../sql-tests/results/kllquantiles.sql.out | 120 ++++++++++++++++++ .../spark/sql/DataFrameAggregateSuite.scala | 7 +- 5 files changed, 305 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala index 18a9fc6e1f19f..0556ef118e02d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.datasketches.kll.{KllDoublesSketch, KllFloatsSketch, KllLongsSketch} import org.apache.datasketches.memory.Memory +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.Cast.{toSQLExpr, toSQLId, toSQLType} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.errors.QueryExecutionErrors @@ -466,6 +468,21 @@ abstract class KllSketchGetQuantileBase /** The output data type for a single value (not array) */ protected def outputDataType: DataType + // The rank argument must be foldable (compile-time constant). + // This enables Photon to efficiently handle array outputs with a known constant size. + override def checkInputDataTypes(): TypeCheckResult = { + if (!right.foldable) { + TypeCheckResult.DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId("rank"), + "inputType" -> toSQLType(right.dataType), + "inputExpr" -> toSQLExpr(right))) + } else { + super.checkInputDataTypes() + } + } + override def nullIntolerant: Boolean = true override def inputTypes: Seq[AbstractDataType] = Seq( @@ -485,7 +502,7 @@ abstract class KllSketchGetQuantileBase val buffer = leftInput.asInstanceOf[Array[Byte]] val memory = Memory.wrap(buffer) - right.eval() match { + rightInput match { case null => null case num: Double => // Single value case @@ -617,6 +634,21 @@ abstract class KllSketchGetRankBase */ protected def kllSketchGetRank(memory: Memory, quantile: Any): Double + // The quantile argument must be foldable (compile-time constant). + // This enables Photon to efficiently handle array outputs with a known constant size. + override def checkInputDataTypes(): TypeCheckResult = { + if (!right.foldable) { + TypeCheckResult.DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId("quantile"), + "inputType" -> toSQLType(right.dataType), + "inputExpr" -> toSQLExpr(right))) + } else { + super.checkInputDataTypes() + } + } + override def nullIntolerant: Boolean = true override def inputTypes: Seq[AbstractDataType] = { Seq( @@ -636,7 +668,7 @@ abstract class KllSketchGetRankBase val buffer: Array[Byte] = leftInput.asInstanceOf[Array[Byte]] val memory: Memory = Memory.wrap(buffer) - right.eval() match { + rightInput match { case null => null case value if !value.isInstanceOf[ArrayData] => // Single value case diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out index 64fc8998c9e42..dc22199985f0b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out @@ -1294,6 +1294,118 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, CAST(col1 AS DOUBLE) / 10.0) AS non_foldable_scalar_rank +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"(CAST(col1 AS DOUBLE) / 10.0)\"", + "inputName" : "`rank`", + "inputType" : "\"DOUBLE\"", + "sqlExpr" : "\"kll_sketch_get_quantile_bigint(agg, (CAST(col1 AS DOUBLE) / 10.0))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "kll_sketch_get_quantile_bigint(agg, CAST(col1 AS DOUBLE) / 10.0)" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, array(0.25, CAST(col1 AS DOUBLE) / 10.0, 0.75)) AS non_foldable_array_rank +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"array(0.25, (CAST(col1 AS DOUBLE) / 10.0), 0.75)\"", + "inputName" : "`rank`", + "inputType" : "\"ARRAY\"", + "sqlExpr" : "\"kll_sketch_get_quantile_bigint(agg, array(0.25, (CAST(col1 AS DOUBLE) / 10.0), 0.75))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "kll_sketch_get_quantile_bigint(agg, array(0.25, CAST(col1 AS DOUBLE) / 10.0, 0.75))" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, col1) AS non_foldable_scalar_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col1\"", + "inputName" : "`quantile`", + "inputType" : "\"BIGINT\"", + "sqlExpr" : "\"kll_sketch_get_rank_bigint(agg, col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "kll_sketch_get_rank_bigint(agg, col1)" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, array(1L, col1, 5L)) AS non_foldable_array_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"array(1, col1, 5)\"", + "inputName" : "`quantile`", + "inputType" : "\"ARRAY\"", + "sqlExpr" : "\"kll_sketch_get_rank_bigint(agg, array(1, col1, 5))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "kll_sketch_get_rank_bigint(agg, array(1L, col1, 5L))" + } ] +} + + -- !query DROP TABLE IF EXISTS t_int_1_5_through_7_11 -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql index d0d7fb1f9c127..69d472ac78a60 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql @@ -464,6 +464,41 @@ FROM ( FROM t_double_1_5_through_7_11 ); +-- Negative tests for non-foldable (non-constant) rank/quantile arguments +-- These tests verify that get_quantile and get_rank functions require compile-time constant arguments + +-- Non-foldable scalar rank argument to get_quantile (column reference) +SELECT kll_sketch_get_quantile_bigint(agg, CAST(col1 AS DOUBLE) / 10.0) AS non_foldable_scalar_rank +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +); + +-- Non-foldable array rank argument to get_quantile (array containing column reference) +SELECT kll_sketch_get_quantile_bigint(agg, array(0.25, CAST(col1 AS DOUBLE) / 10.0, 0.75)) AS non_foldable_array_rank +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +); + +-- Non-foldable scalar quantile argument to get_rank (column reference) +SELECT kll_sketch_get_rank_bigint(agg, col1) AS non_foldable_scalar_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +); + +-- Non-foldable array quantile argument to get_rank (array containing column reference) +SELECT kll_sketch_get_rank_bigint(agg, array(1L, col1, 5L)) AS non_foldable_array_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +); + -- Clean up DROP TABLE IF EXISTS t_int_1_5_through_7_11; DROP TABLE IF EXISTS t_long_1_5_through_7_11; diff --git a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out index 3618c851939e0..6f60f30e5681e 100644 --- a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out @@ -1395,6 +1395,126 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, CAST(col1 AS DOUBLE) / 10.0) AS non_foldable_scalar_rank +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"(CAST(col1 AS DOUBLE) / 10.0)\"", + "inputName" : "`rank`", + "inputType" : "\"DOUBLE\"", + "sqlExpr" : "\"kll_sketch_get_quantile_bigint(agg, (CAST(col1 AS DOUBLE) / 10.0))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "kll_sketch_get_quantile_bigint(agg, CAST(col1 AS DOUBLE) / 10.0)" + } ] +} + + +-- !query +SELECT kll_sketch_get_quantile_bigint(agg, array(0.25, CAST(col1 AS DOUBLE) / 10.0, 0.75)) AS non_foldable_array_rank +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"array(0.25, (CAST(col1 AS DOUBLE) / 10.0), 0.75)\"", + "inputName" : "`rank`", + "inputType" : "\"ARRAY\"", + "sqlExpr" : "\"kll_sketch_get_quantile_bigint(agg, array(0.25, (CAST(col1 AS DOUBLE) / 10.0), 0.75))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "kll_sketch_get_quantile_bigint(agg, array(0.25, CAST(col1 AS DOUBLE) / 10.0, 0.75))" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, col1) AS non_foldable_scalar_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col1\"", + "inputName" : "`quantile`", + "inputType" : "\"BIGINT\"", + "sqlExpr" : "\"kll_sketch_get_rank_bigint(agg, col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "kll_sketch_get_rank_bigint(agg, col1)" + } ] +} + + +-- !query +SELECT kll_sketch_get_rank_bigint(agg, array(1L, col1, 5L)) AS non_foldable_array_quantile +FROM ( + SELECT kll_sketch_agg_bigint(col1) AS agg, col1 + FROM t_long_1_5_through_7_11 + GROUP BY col1 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"array(1, col1, 5)\"", + "inputName" : "`quantile`", + "inputType" : "\"ARRAY\"", + "sqlExpr" : "\"kll_sketch_get_rank_bigint(agg, array(1, col1, 5))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "kll_sketch_get_rank_bigint(agg, array(1L, col1, 5L))" + } ] +} + + -- !query DROP TABLE IF EXISTS t_int_1_5_through_7_11 -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index da2fbceae97e1..0dfd37ebeae00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.time.{Duration, LocalDateTime, LocalTime, Period} +import java.util.Locale import scala.util.Random @@ -3348,7 +3349,7 @@ class DataFrameAggregateSuite extends QueryTest val result = sketchDf.select(kll_sketch_to_string_bigint($"sketch")).collect()(0)(0) assert(result != null) assert(result.asInstanceOf[String].length > 0) - assert(result.asInstanceOf[String].contains("Kll")) + assert(result.asInstanceOf[String].toLowerCase(Locale.ROOT).contains("kll")) } test("kll_sketch_get_n functions") { @@ -3402,7 +3403,7 @@ class DataFrameAggregateSuite extends QueryTest // Test to_string val str = sketchDf.select(kll_sketch_to_string_float($"sketch")).collect()(0)(0) - assert(str.asInstanceOf[String].contains("Kll")) + assert(str.asInstanceOf[String].toLowerCase(Locale.ROOT).contains("kll")) // Test get_n val n = sketchDf.select(kll_sketch_get_n_float($"sketch")).collect()(0)(0) @@ -3433,7 +3434,7 @@ class DataFrameAggregateSuite extends QueryTest // Test to_string val str = sketchDf.select(kll_sketch_to_string_double($"sketch")).collect()(0)(0) - assert(str.asInstanceOf[String].contains("Kll")) + assert(str.asInstanceOf[String].toLowerCase(Locale.ROOT).contains("kll")) // Test get_n val n = sketchDf.select(kll_sketch_get_n_double($"sketch")).collect()(0)(0) From 7426d82c59b2feddf7c1da507f2f340c4cc5a514 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 15 Dec 2025 15:33:30 -0800 Subject: [PATCH 299/400] [SPARK-54686][SQL] Relax DSv2 table checks in temp views to allow new top-level columns ### What changes were proposed in this pull request? This PR relaxes DSv2 table checks in temp views to allow new top-level columns. ### Why are the changes needed? These changes are needed to avoid regressions introduced in upcoming 4.1. The newly added table checks are too strict. We should follow the SQL view behavior and allow new top-level columns. ### Does this PR introduce _any_ user-facing change? Yes but for unreleased functionality. ### How was this patch tested? Existing + new tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53450 from aokolnychyi/spark-54686. Authored-by: Anton Okolnychyi Signed-off-by: Gengliang Wang (cherry picked from commit 2a28bb01ae16d6164733ee741a3116c0f6d22827) Signed-off-by: Gengliang Wang --- .../catalyst/analysis/V2TableReference.scala | 6 +- .../apache/spark/sql/util/SchemaUtils.scala | 5 +- .../connector/catalog/V2TableUtilSuite.scala | 90 ++++++++++++++++++- .../DataSourceV2DataFrameSuite.scala | 81 +++++++++-------- 4 files changed, 139 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala index b6a2c6db66044..85c36d452b309 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/V2TableReference.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.connector.catalog.V2TableUtil import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.SchemaValidationMode.ALLOW_NEW_TOP_LEVEL_FIELDS import org.apache.spark.util.ArrayImplicits._ /** @@ -120,7 +121,10 @@ private[sql] object V2TableReferenceUtils extends SQLConfHelper { ctx: TemporaryViewContext): Unit = { val tableName = ref.identifier.toQualifiedNameParts(ref.catalog) - val dataErrors = V2TableUtil.validateCapturedColumns(table, ref.info.columns) + val dataErrors = V2TableUtil.validateCapturedColumns( + table, + ref.info.columns, + mode = ALLOW_NEW_TOP_LEVEL_FIELDS) if (dataErrors.nonEmpty) { throw QueryCompilationErrors.columnsChangedAfterViewWithPlanCreation( ctx.viewName, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 76c4d518df6f0..58ababa04739f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdenti import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES +import org.apache.spark.sql.util.SchemaValidationMode.{ALLOW_NEW_TOP_LEVEL_FIELDS, PROHIBIT_CHANGES} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkSchemaUtils @@ -459,7 +459,7 @@ private[spark] object SchemaUtils { } } - if (mode == PROHIBIT_CHANGES) { + if (mode == PROHIBIT_CHANGES || (mode == ALLOW_NEW_TOP_LEVEL_FIELDS && colPath.nonEmpty)) { otherFieldsByName.foreach { case (normalizedName, otherField) => if (!fieldsByName.contains(normalizedName)) { errors += s"${formatField(colPath, otherField)} has been added" @@ -529,4 +529,5 @@ private[spark] sealed trait SchemaValidationMode private[spark] object SchemaValidationMode { case object PROHIBIT_CHANGES extends SchemaValidationMode case object ALLOW_NEW_FIELDS extends SchemaValidationMode + case object ALLOW_NEW_TOP_LEVEL_FIELDS extends SchemaValidationMode } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala index b4923b1169371..a9e5668d7fefb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/V2TableUtilSuite.scala @@ -26,7 +26,8 @@ import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.sql.util.SchemaValidationMode.PROHIBIT_CHANGES +import org.apache.spark.sql.util.SchemaValidationMode.{ALLOW_NEW_TOP_LEVEL_FIELDS, PROHIBIT_CHANGES} +import org.apache.spark.sql.util.SchemaValidationMode import org.apache.spark.util.ArrayImplicits.SparkArrayOps class V2TableUtilSuite extends SparkFunSuite { @@ -529,6 +530,86 @@ class V2TableUtilSuite extends SparkFunSuite { assert(errors.head == "`person`.`attrs`.`value` type has changed from INT to BIGINT") } + test("validateCapturedColumns - ALLOW_NEW_TOP_LEVEL_FIELDS allows top-level additions") { + val originCols = Array( + col("id", LongType, nullable = false), + col("name", StringType, nullable = true)) + val currentCols = Array( + col("id", LongType, nullable = false), + col("name", StringType, nullable = true), + col("age", IntegerType, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols, ALLOW_NEW_TOP_LEVEL_FIELDS) + assert(errors.isEmpty) + } + + test("validateCapturedColumns - ALLOW_NEW_TOP_LEVEL_FIELDS prohibits nested additions") { + val originAddress = StructType(Seq( + StructField("street", StringType), + StructField("city", StringType))) + val originCols = Array( + col("id", LongType, nullable = false), + col("address", originAddress, nullable = true)) + val currentAddress = StructType(Seq( + StructField("street", StringType), + StructField("city", StringType), + StructField("zipCode", StringType))) + val currentCols = Array( + col("id", LongType, nullable = false), + col("address", currentAddress, nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols, ALLOW_NEW_TOP_LEVEL_FIELDS) + assert(errors.size == 1) + assert(errors.head.contains("`address`.`zipCode` STRING has been added")) + } + + test("validateCapturedColumns - ALLOW_NEW_TOP_LEVEL_FIELDS fails new nested fields in array") { + val originItem = StructType(Seq( + StructField("itemId", LongType), + StructField("itemName", StringType))) + val originCols = Array( + col("id", LongType, nullable = false), + col("items", ArrayType(originItem), nullable = true)) + val currentItem = StructType(Seq( + StructField("itemId", LongType), + StructField("itemName", StringType), + StructField("price", IntegerType))) + val currentCols = Array( + col("id", LongType, nullable = false), + col("items", ArrayType(currentItem), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = V2TableUtil.validateCapturedColumns( + table, + originCols.toImmutableArraySeq, + mode = ALLOW_NEW_TOP_LEVEL_FIELDS) + assert(errors.size == 1) + assert(errors.head.contains("`items`.`element`.`price` INT has been added")) + } + + test("validateCapturedColumns - ALLOW_NEW_TOP_LEVEL_FIELDS prohibits nested map additions") { + val originValue = StructType(Seq( + StructField("count", IntegerType), + StructField("status", StringType))) + val originCols = Array( + col("id", LongType, nullable = false), + col("metadata", MapType(StringType, originValue), nullable = true)) + val currentValue = StructType(Seq( + StructField("count", IntegerType), + StructField("status", StringType), + StructField("timestamp", LongType))) + val currentCols = Array( + col("id", LongType, nullable = false), + col("metadata", MapType(StringType, currentValue), nullable = true)) + val table = TestTableWithMetadataSupport("test", currentCols) + + val errors = validateCapturedColumns(table, originCols, ALLOW_NEW_TOP_LEVEL_FIELDS) + assert(errors.size == 1) + assert(errors.head.contains("`metadata`.`value`.`timestamp` BIGINT has been added")) + } + // simple table without metadata column support private case class TestTable( override val name: String, @@ -555,8 +636,11 @@ class V2TableUtilSuite extends SparkFunSuite { override def metadataInJSON: String = "{}" } - private def validateCapturedColumns(table: Table, originCols: Array[Column]): Seq[String] = { - V2TableUtil.validateCapturedColumns(table, originCols.toImmutableArraySeq) + private def validateCapturedColumns( + table: Table, + originCols: Array[Column], + mode: SchemaValidationMode = PROHIBIT_CHANGES): Seq[String] = { + V2TableUtil.validateCapturedColumns(table, originCols.toImmutableArraySeq, mode) } private def col(name: String, dataType: DataType, nullable: Boolean): Column = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index cdb498c8f2bf8..c3164b3428f95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -1254,7 +1254,7 @@ class DataSourceV2DataFrameSuite } } - test("SPARK-54157: cached temp view detects schema changes after analysis") { + test("SPARK-54157: cached temp view allows top-level column additions") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id INT, data STRING) USING foo") @@ -1268,15 +1268,9 @@ class DataSourceV2DataFrameSuite // change table schema after the view has been analyzed and cached sql(s"ALTER TABLE $t ADD COLUMN extra INT") - // execution should fail with column mismatch even though the view is cached - checkError( - exception = intercept[AnalysisException] { spark.table("v").collect() }, - condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", - parameters = Map( - "viewName" -> "`v`", - "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "colType" -> "data", - "errors" -> "- `extra` INT has been added")) + // execution should succeed as top-level column additions are allowed + // the temp view captures the original columns just like SQL views + checkAnswer(spark.table("v"), Seq(Row(1, "a"))) } } @@ -1512,7 +1506,7 @@ class DataSourceV2DataFrameSuite } } - test("SPARK-53924: temp view on DSv2 table detects added columns") { + test("SPARK-53924: temp view on DSv2 table allows top-level column additions") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id bigint, data string) USING foo") @@ -1521,10 +1515,32 @@ class DataSourceV2DataFrameSuite spark.table(t).createOrReplaceTempView("v") checkAnswer(spark.table("v"), Seq.empty) - // add column to underlying table + // add top-level column to underlying table sql(s"ALTER TABLE $t ADD COLUMN age int") - // accessing temp view should detect schema change + // accessing temp view should succeed as top-level column additions are allowed + // view captures original columns + checkAnswer(spark.table("v"), Seq.empty) + + // insert data to verify view still works correctly + sql(s"INSERT INTO $t VALUES (1, 'a', 25)") + checkAnswer(spark.table("v"), Seq(Row(1, "a"))) + } + } + + test("SPARK-53924: temp view on DSv2 table detects nested column additions") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, address STRUCT) USING foo") + + // create temp view using DataFrame API + spark.table(t).createOrReplaceTempView("v") + checkAnswer(spark.table("v"), Seq.empty) + + // add nested column to underlying table + sql(s"ALTER TABLE $t ADD COLUMN address.zipCode string") + + // accessing temp view should detect schema change for nested additions checkError( exception = intercept[AnalysisException] { spark.table("v").collect() }, condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", @@ -1532,7 +1548,7 @@ class DataSourceV2DataFrameSuite "viewName" -> "`v`", "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", "colType" -> "data", - "errors" -> "- `age` INT has been added")) + "errors" -> "- `address`.`zipCode` STRING has been added")) } } @@ -1620,13 +1636,13 @@ class DataSourceV2DataFrameSuite test("SPARK-53924: createOrReplaceTempView works after schema change") { val t = "testcat.ns1.ns2.tbl" withTable(t) { - sql(s"CREATE TABLE $t (id bigint) USING foo") + sql(s"CREATE TABLE $t (id bigint, data STRING, extra INT) USING foo") spark.table(t).createOrReplaceTempView("v") checkAnswer(spark.table("v"), Seq.empty) // alter table - sql(s"ALTER TABLE $t ADD COLUMN data string") + sql(s"ALTER TABLE $t DROP COLUMN extra") // old view fails intercept[AnalysisException] { spark.table("v").collect() } @@ -1658,22 +1674,16 @@ class DataSourceV2DataFrameSuite }.get assert(options.get("fakeOption") == "testValue") - // schema changes should still be detected + // add top-level column to underlying table sql(s"ALTER TABLE $t ADD COLUMN age int") - // accessing temp view should detect schema change - checkError( - exception = intercept[AnalysisException] { spark.table("v").collect() }, - condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", - parameters = Map( - "viewName" -> "`v`", - "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "colType" -> "data", - "errors" -> "- `age` INT has been added")) + // accessing temp view should succeed as top-level column additions are allowed + + checkAnswer(spark.table("v"), Seq.empty) } } - test("SPARK-53924: temp view on DSv2 table created using SQL with plan detects changes") { + test("SPARK-53924: temp view on DSv2 table created using SQL with plan and top-level additions") { val t = "testcat.ns1.ns2.tbl" withTable(t) { withSQLConf(SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW.key -> "true") { @@ -1687,18 +1697,15 @@ class DataSourceV2DataFrameSuite val Some(view) = spark.sessionState.catalog.getRawTempView("v") assert(view.plan.isDefined) - // add column to underlying table + // add top-level column to underlying table sql(s"ALTER TABLE $t ADD COLUMN age int") - // accessing temp view should detect schema change - checkError( - exception = intercept[AnalysisException] { spark.table("v").collect() }, - condition = "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION", - parameters = Map( - "viewName" -> "`v`", - "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", - "colType" -> "data", - "errors" -> "- `age` INT has been added")) + // accessing temp view should succeed as top-level column additions are allowed + checkAnswer(spark.table("v"), Seq.empty) + + // insert data to verify view still works correctly + sql(s"INSERT INTO $t VALUES (1, 'a', 25)") + checkAnswer(spark.table("v"), Seq(Row(1, "a"))) } } } From 37ec5c4fab2d0f20139ac953008e616d3f3f2858 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Herman=20van=20H=C3=B6vell?= Date: Tue, 16 Dec 2025 08:44:00 +0900 Subject: [PATCH 300/400] [SPARK-54696][CONNECT] Clean-up ArrowBuffers in Connect MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR fixes a memory leak in Spark Connect LocalRelations. ... more details TBD ... ### Why are the changes needed? It fixes a stability issue. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. A Connect Planner Test TBD Longevity tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53452 from hvanhovell/fix-arrow-local-relations. Authored-by: Herman van Hövell Signed-off-by: Dongjoon Hyun (cherry picked from commit c36b7e58d0422a13228252657e4cff26a762a228) Signed-off-by: Dongjoon Hyun --- .../spark/sql/util}/CloseableIterator.scala | 2 +- .../ConcatenatingArrowStreamReader.scala | 8 +- .../client/arrow/ArrowEncoderSuite.scala | 2 +- .../spark/sql/connect/SparkSession.scala | 4 +- .../connect/StreamingQueryListenerBus.scala | 2 +- .../CustomSparkConnectBlockingStub.scala | 1 + ...cutePlanResponseReattachableIterator.scala | 1 + .../client/GrpcExceptionConverter.scala | 1 + .../sql/connect/client/GrpcRetryHandler.scala | 1 + .../connect/client/ResponseValidator.scala | 1 + .../connect/client/SparkConnectClient.scala | 1 + .../sql/connect/client/SparkResult.scala | 4 +- .../client/arrow/ArrowDeserializer.scala | 2 +- .../client/arrow/ArrowSerializer.scala | 3 +- .../connect/planner/SparkConnectPlanner.scala | 38 +++----- .../sql/connect/SparkConnectServerTest.scala | 3 +- .../sql/execution/arrow/ArrowConverters.scala | 96 ++++++++----------- .../arrow/ArrowConvertersSuite.scala | 16 ++-- 18 files changed, 83 insertions(+), 103 deletions(-) rename sql/{connect/common/src/main/scala/org/apache/spark/sql/connect/client => api/src/main/scala/org/apache/spark/sql/util}/CloseableIterator.scala (97%) rename sql/{connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow => api/src/main/scala/org/apache/spark/sql/util}/ConcatenatingArrowStreamReader.scala (95%) diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/CloseableIterator.scala similarity index 97% rename from sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala rename to sql/api/src/main/scala/org/apache/spark/sql/util/CloseableIterator.scala index 9de585503a500..dc38c75d3ce73 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/CloseableIterator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.connect.client +package org.apache.spark.sql.util private[sql] trait CloseableIterator[E] extends Iterator[E] with AutoCloseable { self => def asJava: java.util.Iterator[E] = new java.util.Iterator[E] with AutoCloseable { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ConcatenatingArrowStreamReader.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala similarity index 95% rename from sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ConcatenatingArrowStreamReader.scala rename to sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala index 90963c831c252..5de53a568a7d3 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ConcatenatingArrowStreamReader.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.connect.client.arrow +package org.apache.spark.sql.util import java.io.{InputStream, IOException} import java.nio.channels.Channels @@ -34,7 +34,7 @@ import org.apache.arrow.vector.types.pojo.Schema * closes its messages when it consumes them. In order to prevent that from happening in * non-destructive mode we clone the messages before passing them to the reading logic. */ -class ConcatenatingArrowStreamReader( +private[sql] class ConcatenatingArrowStreamReader( allocator: BufferAllocator, input: Iterator[AbstractMessageIterator], destructive: Boolean) @@ -128,7 +128,7 @@ class ConcatenatingArrowStreamReader( override def closeReadSource(): Unit = () } -trait AbstractMessageIterator extends Iterator[ArrowMessage] { +private[sql] trait AbstractMessageIterator extends Iterator[ArrowMessage] { def schema: Schema def bytesRead: Long } @@ -137,7 +137,7 @@ trait AbstractMessageIterator extends Iterator[ArrowMessage] { * Decode an Arrow IPC stream into individual messages. Please note that this iterator MUST have a * valid IPC stream as its input, otherwise construction will fail. */ -class MessageIterator(input: InputStream, allocator: BufferAllocator) +private[sql] class MessageIterator(input: InputStream, allocator: BufferAllocator) extends AbstractMessageIterator { private[this] val in = new ReadChannel(Channels.newChannel(input)) private[this] val reader = new MessageChannelReader(in, allocator) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala index d24369ff5fc7c..52a503d626018 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala @@ -41,10 +41,10 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_SECOND import org.apache.spark.sql.catalyst.util.IntervalStringStyles.ANSI_STYLE import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils._ import org.apache.spark.sql.catalyst.util.SparkIntervalUtils._ -import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.connect.client.arrow.FooEnum.FooEnum import org.apache.spark.sql.connect.test.ConnectFunSuite import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, Geography, Geometry, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType} +import org.apache.spark.sql.util.CloseableIterator import org.apache.spark.unsafe.types.VariantVal import org.apache.spark.util.{MaybeNull, SparkStringUtils} diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala index daa2cc2001e42..42dd1a2b99793 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala @@ -51,13 +51,13 @@ import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{agnosticEncoderFor, BoxedLongEncoder, UnboundRowEncoder} import org.apache.spark.sql.connect.ColumnNodeToProtoConverter.toLiteral import org.apache.spark.sql.connect.ConnectConversions._ -import org.apache.spark.sql.connect.client.{ClassFinder, CloseableIterator, SparkConnectClient, SparkResult} +import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.client.SparkConnectClient.Configuration import org.apache.spark.sql.connect.client.arrow.ArrowSerializer import org.apache.spark.sql.internal.{SessionState, SharedState, SqlApiConf, SubqueryExpression} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.ExecutionListenerManager +import org.apache.spark.sql.util.{CloseableIterator, ExecutionListenerManager} import org.apache.spark.util.ArrayImplicits._ /** diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala index 52b0ea24e9e33..7548238351468 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala @@ -23,9 +23,9 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.connect.proto.{Command, ExecutePlanResponse, Plan, StreamingQueryEventType} import org.apache.spark.internal.{Logging, LogKeys} -import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.streaming.StreamingQueryListener import org.apache.spark.sql.streaming.StreamingQueryListener.{Event, QueryIdleEvent, QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent} +import org.apache.spark.sql.util.CloseableIterator class StreamingQueryListenerBus(sparkSession: SparkSession) extends Logging { private val listeners = new CopyOnWriteArrayList[StreamingQueryListener]() diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala index 913f068fcf345..715da0df73491 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala @@ -21,6 +21,7 @@ import scala.jdk.CollectionConverters._ import io.grpc.ManagedChannel import org.apache.spark.connect.proto._ +import org.apache.spark.sql.util.CloseableIterator private[connect] class CustomSparkConnectBlockingStub( channel: ManagedChannel, diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala index f3c13c9c2c4d8..131a2e77cc431 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala @@ -28,6 +28,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.connect.proto import org.apache.spark.internal.Logging import org.apache.spark.sql.connect.client.GrpcRetryHandler.RetryException +import org.apache.spark.sql.util.WrappedCloseableIterator /** * Retryable iterator of ExecutePlanResponses to an ExecutePlan call. diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala index d3dae47f4c471..7e0b0949fcf1d 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.streaming.StreamingQueryException +import org.apache.spark.sql.util.{CloseableIterator, WrappedCloseableIterator} import org.apache.spark.util.ArrayImplicits._ /** diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala index 3f4558ee97dad..d92dc902fedc5 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala @@ -23,6 +23,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, NUM_RETRY, POLICY, RETRY_WAIT_TIME} +import org.apache.spark.sql.util.{CloseableIterator, WrappedCloseableIterator} private[sql] class GrpcRetryHandler( private val policies: Seq[RetryPolicy], diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala index 03548120457f3..6cf39b8d18798 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala @@ -23,6 +23,7 @@ import io.grpc.{Status, StatusRuntimeException} import io.grpc.stub.StreamObserver import org.apache.spark.internal.Logging +import org.apache.spark.sql.util.{CloseableIterator, WrappedCloseableIterator} // This is common logic to be shared between different stub instances to keep the server-side // session id and to validate responses as seen by the client. diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index 1a7d062470e1c..5d36fc45f9480 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -40,6 +40,7 @@ import org.apache.spark.internal.LogKeys.{ERROR, RATIO, SIZE, TIME} import org.apache.spark.sql.connect.RuntimeConfig import org.apache.spark.sql.connect.common.ProtoUtils import org.apache.spark.sql.connect.common.config.ConnectCommon +import org.apache.spark.sql.util.CloseableIterator import org.apache.spark.util.SparkSystemUtils /** diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 43265e55a0ca9..4199801d8505c 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -35,10 +35,10 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator} +import org.apache.spark.sql.connect.client.arrow.ArrowDeserializingIterator import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter} import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.util.{AbstractMessageIterator, ArrowUtils, CloseableIterator, ConcatenatingArrowStreamReader, MessageIterator} private[sql] class SparkResult[T]( responses: CloseableIterator[proto.ExecutePlanResponse], diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index 8d5811dda8f35..82029025a7f0b 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -37,9 +37,9 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.errors.{CompilationErrors, ExecutionErrors} import org.apache.spark.sql.types.Decimal +import org.apache.spark.sql.util.{CloseableIterator, ConcatenatingArrowStreamReader, MessageIterator} import org.apache.spark.unsafe.types.VariantVal /** diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala index 73c9a991ab6a7..d547c81afe5ad 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala @@ -38,10 +38,9 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, Codec} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.util.{SparkDateTimeUtils, SparkIntervalUtils} -import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.errors.ExecutionErrors import org.apache.spark.sql.types.Decimal -import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.util.{ArrowUtils, CloseableIterator} import org.apache.spark.unsafe.types.VariantVal /** diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 9af2e7cb46616..9cbb760f6cc0e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -30,7 +30,7 @@ import com.google.protobuf.{Any => ProtoAny, ByteString, Message} import io.grpc.{Context, Status, StatusRuntimeException} import io.grpc.stub.StreamObserver -import org.apache.spark.{SparkClassNotFoundException, SparkEnv, SparkException, TaskContext} +import org.apache.spark.{SparkClassNotFoundException, SparkEnv, SparkException} import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction} import org.apache.spark.connect.proto @@ -1492,9 +1492,12 @@ class SparkConnectPlanner( } if (rel.hasData) { - val (rows, structType) = - ArrowConverters.fromIPCStream(rel.getData.toByteArray, TaskContext.get()) - buildLocalRelationFromRows(rows, structType, Option(schema)) + val (rows, structType) = ArrowConverters.fromIPCStream(rel.getData.toByteArray) + try { + buildLocalRelationFromRows(rows, structType, Option(schema)) + } finally { + rows.close() + } } else { if (schema == null) { throw InvalidInputErrors.schemaRequiredForLocalRelation() @@ -1565,28 +1568,13 @@ class SparkConnectPlanner( } // Load and combine all batches - var combinedRows: Iterator[InternalRow] = Iterator.empty - var structType: StructType = null - - for ((dataHash, batchIndex) <- dataHashes.zipWithIndex) { - val dataBytes = readChunkedCachedLocalRelationBlock(dataHash) - val (batchRows, batchStructType) = - ArrowConverters.fromIPCStream(dataBytes, TaskContext.get()) - - // For the first batch, set the schema; for subsequent batches, verify compatibility - if (batchIndex == 0) { - structType = batchStructType - combinedRows = batchRows - - } else { - if (batchStructType != structType) { - throw InvalidInputErrors.chunkedCachedLocalRelationChunksWithDifferentSchema() - } - combinedRows = combinedRows ++ batchRows - } + val (rows, structType) = + ArrowConverters.fromIPCStream(dataHashes.iterator.map(readChunkedCachedLocalRelationBlock)) + try { + buildLocalRelationFromRows(rows, structType, Option(schema)) + } finally { + rows.close() } - - buildLocalRelationFromRows(combinedRows, structType, Option(schema)) } private def toStructTypeOrWrap(dt: DataType): StructType = dt match { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala index 7b9052bb9d2c9..77ede8e852e87 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.classic import org.apache.spark.sql.connect -import org.apache.spark.sql.connect.client.{CloseableIterator, CustomSparkConnectBlockingStub, ExecutePlanResponseReattachableIterator, RetryPolicy, SparkConnectClient, SparkConnectStubState} +import org.apache.spark.sql.connect.client.{CustomSparkConnectBlockingStub, ExecutePlanResponseReattachableIterator, RetryPolicy, SparkConnectClient, SparkConnectStubState} import org.apache.spark.sql.connect.client.arrow.ArrowSerializer import org.apache.spark.sql.connect.common.config.ConnectCommon import org.apache.spark.sql.connect.config.Connect @@ -38,6 +38,7 @@ import org.apache.spark.sql.connect.dsl.MockRemoteSession import org.apache.spark.sql.connect.dsl.plans._ import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionKey, SparkConnectService} import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.util.CloseableIterator /** * Base class and utilities for a test suite that starts and tests the real SparkConnectService diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 8b031af14e8b1..e227a58528721 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -22,6 +22,7 @@ import java.nio.channels.{Channels, ReadableByteChannel} import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ +import scala.util.control.NonFatal import org.apache.arrow.compression.{Lz4CompressionCodec, ZstdCompressionCodec} import org.apache.arrow.flatbuf.MessageHeader @@ -42,12 +43,11 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.util.{ArrowUtils, CloseableIterator, ConcatenatingArrowStreamReader, MessageIterator} import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.util.{ByteBufferOutputStream, SizeEstimator, Utils} import org.apache.spark.util.ArrayImplicits._ - /** * Writes serialized ArrowRecordBatches to a DataOutputStream in the Arrow stream format. */ @@ -297,50 +297,33 @@ private[sql] object ArrowConverters extends Logging { * @param context Task Context for Spark */ private[sql] class InternalRowIteratorFromIPCStream( - input: Array[Byte], - context: TaskContext) extends Iterator[InternalRow] { - - // Keep all the resources we have opened in order, should be closed - // in reverse order finally. - private val resources = new ArrayBuffer[AutoCloseable]() + ipcStreams: Iterator[Array[Byte]], + context: TaskContext) + extends CloseableIterator[InternalRow] { // Create an allocator used for all Arrow related memory. protected val allocator: BufferAllocator = ArrowUtils.rootAllocator.newChildAllocator( s"to${this.getClass.getSimpleName}", 0, Long.MaxValue) - resources.append(allocator) - private val reader = try { - new ArrowStreamReader(new ByteArrayInputStream(input), allocator) - } catch { - case e: Exception => - closeAll(resources.toSeq.reverse: _*) - throw new IllegalArgumentException( - s"Failed to create ArrowStreamReader: ${e.getMessage}", e) - } - resources.append(reader) - - private val root: VectorSchemaRoot = try { - reader.getVectorSchemaRoot - } catch { - case e: Exception => - closeAll(resources.toSeq.reverse: _*) - throw new IllegalArgumentException( - s"Failed to read schema from IPC stream: ${e.getMessage}", e) + private val reader = { + val messages = ipcStreams.map { bytes => + new MessageIterator(new ByteArrayInputStream(bytes), allocator) + } + new ConcatenatingArrowStreamReader(allocator, messages, destructive = true) } - resources.append(root) val schema: StructType = try { - ArrowUtils.fromArrowSchema(root.getSchema) + ArrowUtils.fromArrowSchema(reader.getVectorSchemaRoot.getSchema) } catch { - case e: Exception => - closeAll(resources.toSeq.reverse: _*) - throw new IllegalArgumentException(s"Failed to convert Arrow schema: ${e.getMessage}", e) + case NonFatal(e) => + // Since this triggers a read (which involves allocating buffers) we have to clean-up. + close() + throw e } - // TODO: wrap in exception - private var rowIterator: Iterator[InternalRow] = vectorSchemaRootToIter(root) + private var rowIterator: Iterator[InternalRow] = Iterator.empty // Metrics to track batch processing private var _batchesLoaded: Int = 0 @@ -348,7 +331,7 @@ private[sql] object ArrowConverters extends Logging { if (context != null) { context.addTaskCompletionListener[Unit] { _ => - closeAll(resources.toSeq.reverse: _*) + close() } } @@ -356,28 +339,17 @@ private[sql] object ArrowConverters extends Logging { def batchesLoaded: Int = _batchesLoaded def totalRowsProcessed: Long = _totalRowsProcessed - // Loads the next batch from the Arrow reader and returns true or - // false if the next batch could be loaded. - private def loadNextBatch(): Boolean = { - if (reader.loadNextBatch()) { - rowIterator = vectorSchemaRootToIter(root) - _batchesLoaded += 1 - true - } else { - false - } - } - override def hasNext: Boolean = { - if (rowIterator.hasNext) { - true - } else { - if (!loadNextBatch()) { - false + while (!rowIterator.hasNext) { + if (reader.loadNextBatch()) { + rowIterator = vectorSchemaRootToIter(reader.getVectorSchemaRoot) + _batchesLoaded += 1 } else { - hasNext + close() + return false } } + true } override def next(): InternalRow = { @@ -387,6 +359,10 @@ private[sql] object ArrowConverters extends Logging { _totalRowsProcessed += 1 rowIterator.next() } + + override def close(): Unit = { + closeAll(reader, allocator) + } } /** @@ -512,15 +488,21 @@ private[sql] object ArrowConverters extends Logging { * one schema and a varying number of record batches. Returns an iterator over the * created InternalRow. */ - private[sql] def fromIPCStream(input: Array[Byte], context: TaskContext): - (Iterator[InternalRow], StructType) = { - fromIPCStreamWithIterator(input, context) + private[sql] def fromIPCStream(input: Array[Byte]): + (CloseableIterator[InternalRow], StructType) = { + fromIPCStream(Iterator.single(input)) + } + + private[sql] def fromIPCStream(inputs: Iterator[Array[Byte]]): + (CloseableIterator[InternalRow], StructType) = { + val iterator = new InternalRowIteratorFromIPCStream(inputs, null) + (iterator, iterator.schema) } // Overloaded method for tests to access the iterator with metrics private[sql] def fromIPCStreamWithIterator(input: Array[Byte], context: TaskContext): - (InternalRowIteratorFromIPCStream, StructType) = { - val iterator = new InternalRowIteratorFromIPCStream(input, context) + (InternalRowIteratorFromIPCStream, StructType) = { + val iterator = new InternalRowIteratorFromIPCStream(Iterator.single(input), context) (iterator, iterator.schema) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala index ccf6b63eb5ded..f58a5b7ebd6a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala @@ -1626,7 +1626,8 @@ class ArrowConvertersSuite extends SharedSparkSession { writer.end() } - val (outputRowIter, outputSchema) = ArrowConverters.fromIPCStream(out.toByteArray, ctx) + val (outputRowIter, outputSchema) = ArrowConverters. + fromIPCStreamWithIterator(out.toByteArray, ctx) assert(outputSchema == schema) val res = outputRowIter.zipWithIndex.map { case (row, i) => assert(row.getInt(0) == i) @@ -1663,7 +1664,8 @@ class ArrowConvertersSuite extends SharedSparkSession { writer.end() } - val (outputRowIter, outputSchema) = ArrowConverters.fromIPCStream(out.toByteArray, ctx) + val (outputRowIter, outputSchema) = ArrowConverters + .fromIPCStreamWithIterator(out.toByteArray, ctx) assert(outputSchema == schema) val outputRows = outputRowIter.zipWithIndex.map { case (row, i) => assert(row.getInt(0) == i) @@ -1760,7 +1762,7 @@ class ArrowConvertersSuite extends SharedSparkSession { val invalidData = Array[Byte](1, 2, 3, 4, 5) intercept[Exception] { - ArrowConverters.fromIPCStream(invalidData, ctx) + ArrowConverters.fromIPCStreamWithIterator(invalidData, ctx) } } @@ -1769,7 +1771,7 @@ class ArrowConvertersSuite extends SharedSparkSession { val emptyData = Array.empty[Byte] intercept[Exception] { - ArrowConverters.fromIPCStream(emptyData, ctx) + ArrowConverters.fromIPCStreamWithIterator(emptyData, ctx) } } @@ -1790,7 +1792,8 @@ class ArrowConvertersSuite extends SharedSparkSession { // Test with null context - should still work but won't have cleanup registration val proj = UnsafeProjection.create(schema) - val (outputRowIter, outputSchema) = ArrowConverters.fromIPCStream(out.toByteArray, null) + val (outputRowIter, outputSchema) = ArrowConverters. + fromIPCStreamWithIterator(out.toByteArray, null) assert(outputSchema == schema) val outputRows = outputRowIter.map(proj(_).copy()).toList assert(outputRows.length == inputRows.length) @@ -1884,7 +1887,8 @@ class ArrowConvertersSuite extends SharedSparkSession { writer.end() } - val (outputRowIter, outputSchema) = ArrowConverters.fromIPCStream(out.toByteArray, ctx) + val (outputRowIter, outputSchema) = ArrowConverters. + fromIPCStreamWithIterator(out.toByteArray, ctx) val proj = UnsafeProjection.create(schema) assert(outputSchema == schema) val outputRows = outputRowIter.map(proj(_).copy()).toList From 9e0a14b7a8657d7085a9acc70f5f1fe73a1b3c3e Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 16 Dec 2025 08:57:06 +0900 Subject: [PATCH 301/400] [SPARK-53991][SQL][FOLLOWUP] Remove wrong comments added in PR #53463 ### What changes were proposed in this pull request? This PR removes two wrong comment lines added in [PR #53463](https://github.com/apache/spark/pull/53463). ### Why are the changes needed? These two accidental comment lines don't apply to Apache Spark and will be removed. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A, comment changes only ### Was this patch authored or co-authored using generative AI tooling? No Closes #53477 from dtenedor/kll-get-quantile-rank-remove-comments. Authored-by: Daniel Tenedorio Signed-off-by: Dongjoon Hyun (cherry picked from commit bd46e21a2cb4d30eb9a9028c84807403db190fc6) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/expressions/kllExpressions.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala index 0556ef118e02d..b446b2d0d4433 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/kllExpressions.scala @@ -469,7 +469,6 @@ abstract class KllSketchGetQuantileBase protected def outputDataType: DataType // The rank argument must be foldable (compile-time constant). - // This enables Photon to efficiently handle array outputs with a known constant size. override def checkInputDataTypes(): TypeCheckResult = { if (!right.foldable) { TypeCheckResult.DataTypeMismatch( @@ -635,7 +634,6 @@ abstract class KllSketchGetRankBase protected def kllSketchGetRank(memory: Memory, quantile: Any): Double // The quantile argument must be foldable (compile-time constant). - // This enables Photon to efficiently handle array outputs with a known constant size. override def checkInputDataTypes(): TypeCheckResult = { if (!right.foldable) { TypeCheckResult.DataTypeMismatch( From fea8362746af13e235adc64b9c1c5fd6f9faa86b Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 17 Dec 2025 17:17:54 +0800 Subject: [PATCH 302/400] [SPARK-54728][PYTHON][DOCS] Remove a wrong note in `dataframe.isEmpty` ### What changes were proposed in this pull request? Remove a wrong note in `dataframe.isEmpty` ### Why are the changes needed? > Unlike `count()`, this method does not trigger any computation. this note is incorrect, a computation will be triggered in most cases ### Does this PR introduce _any_ user-facing change? yes, doc-only change ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #53500 from zhengruifeng/doc_fix_is_empty. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit 10bc8bcba103b3f268e5b929291ae140af94754d) Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/dataframe.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 502883cf59b1a..f8f5cfecd2c65 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -852,7 +852,6 @@ def isEmpty(self) -> bool: Notes ----- - - Unlike `count()`, this method does not trigger any computation. - An empty DataFrame has no rows. It may have columns, but no data. Examples From 0125dc4ceac2bfae203709bab3a1473dced2377f Mon Sep 17 00:00:00 2001 From: Kristin Cowalcijk Date: Wed, 17 Dec 2025 11:12:49 -0400 Subject: [PATCH 303/400] [SPARK-52819][SQL] Making KryoSerializationCodec serializable to fix java.io.NotSerializableException errors in various use cases MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR makes `KryoSerializationCodec` implements `java.io.Serializable` to avoid `java.io.NotSerializableException` exceptions when using Kryo encoder in `Dataset.flatMapGroupsWithState` or `Aggregator.bufferEncoder`. ### Why are the changes needed? See the description in [SPARK-52819](https://issues.apache.org/jira/browse/SPARK-52819) as well as the [minimal repro](https://github.com/Kontinuation/spark-4-kryo-encoder-bug). The problems only happens when using Spark 4.0.0 but not when using Spark 3.5.5. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests were added to ensure that 1. Both kryo encoder and java serialization encoder are serializable 2. The kryo encoded aggregator buffer use case works ### Was this patch authored or co-authored using generative AI tooling? No. Closes #51615 from Kontinuation/fix-kryo-codec-serializable. Authored-by: Kristin Cowalcijk Signed-off-by: Herman van Hövell (cherry picked from commit e3569bafd8da992e0d9687075a2255b2cd6eb779) Signed-off-by: Herman van Hövell --- .../spark/sql/catalyst/encoders/codecs.scala | 2 +- .../encoders/ExpressionEncoderSuite.scala | 1 + .../scala/org/apache/spark/sql/UDFSuite.scala | 18 ++++++++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala index 0f21972552339..b90d9f8013d6f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala @@ -55,7 +55,7 @@ object JavaSerializationCodec extends (() => Codec[Any, Array[Byte]]) { * server (driver & executors) very tricky. As a workaround a user can define their own Codec * which internalizes the Kryo configuration. */ -object KryoSerializationCodec extends (() => Codec[Any, Array[Byte]]) { +object KryoSerializationCodec extends (() => Codec[Any, Array[Byte]]) with Serializable { private lazy val kryoCodecConstructor: MethodHandle = { val cls = SparkClassUtils.classForName( "org.apache.spark.sql.catalyst.encoders.KryoSerializationCodecImpl") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index a5c11558b3db3..287b99d10d659 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -612,6 +612,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes provider, nullable = true)) .resolveAndBind() + assert(encoder.isInstanceOf[Serializable]) assert(encoder.schema == new StructType().add("value", BinaryType)) val toRow = encoder.createSerializer() val fromRow = encoder.createDeserializer() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 740ec05994f5e..31ed0f26d9b95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -53,6 +53,17 @@ private case class FunctionResult(f1: String, f2: String) private case class LocalDateInstantType(date: LocalDate, instant: Instant) private case class TimestampInstantType(t: Timestamp, instant: Instant) +private case class KryoEncodedBuf(value: Long) +private case class KryoBufAggregator() extends Aggregator[Long, KryoEncodedBuf, Long] { + override def zero: KryoEncodedBuf = KryoEncodedBuf(0) + override def reduce(b: KryoEncodedBuf, a: Long): KryoEncodedBuf = KryoEncodedBuf(b.value + a) + override def merge(b1: KryoEncodedBuf, b2: KryoEncodedBuf): KryoEncodedBuf = + KryoEncodedBuf(b1.value + b2.value) + override def finish(reduction: KryoEncodedBuf): Long = reduction.value + override def bufferEncoder: Encoder[KryoEncodedBuf] = Encoders.kryo[KryoEncodedBuf] + override def outputEncoder: Encoder[Long] = Encoders.scalaLong +} + class UDFSuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -1249,4 +1260,11 @@ class UDFSuite extends QueryTest with SharedSparkSession { .select(f($"c").as("f"), f($"f")) checkAnswer(df, Seq(Row(2, 3), Row(null, null))) } + + test("SPARK-52819: Support using Kryo to encode BUF in Aggregator") { + val kryoBufUDAF = udaf(KryoBufAggregator()) + val input = Seq(1L, 2L, 3L).toDF("value") + val result = input.select(kryoBufUDAF($"value").as("sum")) + checkAnswer(result, Row(6L) :: Nil) + } } From 9040932f641cf040e8d03657e51a2bef3a096d50 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 18 Dec 2025 08:31:02 +0800 Subject: [PATCH 304/400] [SPARK-54730][SQL][CONNECT] Delay failure of dataframe column resolution ### What changes were proposed in this pull request? Delay failure of dataframe column resolution ### Why are the changes needed? it cause conflicts with delta rules that add hidden column ### Does this PR introduce _any_ user-facing change? yes, delta query fails before this fix ```py df = spark.read.option("readChangeFeed", True).option("startingVersion", 0).table("sample_table") df.select(df._commit_version).show() <- fail with [CANNOT_RESOLVE_DATAFRAME_COLUMN] Cannot resolve dataframe column "_commit_version". ``` ### How was this patch tested? added test ### Was this patch authored or co-authored using generative AI tooling? no Closes #53503 from zhengruifeng/df_col_delay_fail_v2. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit 36f8f1e1003c676969e5108d58fed57bfbba4ddc) Signed-off-by: Ruifeng Zheng --- .../analysis/ColumnResolutionHelper.scala | 14 ++++---- .../sql/SparkSessionExtensionSuite.scala | 34 +++++++++++++++++-- 2 files changed, 39 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 34541a8840cb9..870e033642252 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -140,7 +140,9 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { } matched(ordinal) - case u @ UnresolvedAttribute(nameParts) => + case u @ UnresolvedAttribute(nameParts) + if u.getTagValue(LogicalPlan.PLAN_ID_TAG).isEmpty => + // UnresolvedAttribute with PLAN_ID_TAG should be resolved in resolveDataFrameColumn val result = withPosition(u) { resolveColumnByName(nameParts) .orElse(LiteralFunctionResolution.resolve(nameParts)) @@ -495,8 +497,7 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { // 1. extract the attached plan id from UnresolvedAttribute; // 2. top-down traverse the query plan to find the plan node that matches the plan id; // 3. if can not find the matching node, fails with 'CANNOT_RESOLVE_DATAFRAME_COLUMN'; - // 4, if the matching node is found, but can not resolve the column, also fails with - // 'CANNOT_RESOLVE_DATAFRAME_COLUMN'; + // 4, if the matching node is found, but can not resolve the column, return the original one; // 5, resolve the expression against the target node, the resolved attribute will be // filtered by the output attributes of nodes in the path (from matching to root node); // 6. if more than one resolved attributes are found in the above recursive process, @@ -571,10 +572,9 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { } else { None } - if (resolved.isEmpty) { - // The targe plan node is found, but the column cannot be resolved. - throw QueryCompilationErrors.cannotResolveDataFrameColumn(u) - } + // The targe plan node is found, but might still fail to resolve. + // In this case, return None to delay the failure, so it is possible to be + // resolved in the next iteration. (resolved.map(r => (r, currentDepth)), true) } else { val children = p match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 6ee0029b68395..66826a9ca762a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -26,18 +26,19 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, Max, Partial} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.{PlanTest, SQLHelper} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, AggregateHint, ColumnStat, Limit, LocalRelation, LogicalPlan, Sort, SortHint, Statistics, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, AggregateHint, ColumnStat, Limit, LocalRelation, LogicalPlan, Project, Sort, SortHint, Statistics, UnresolvedHint} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.classic.ClassicConversions._ +import org.apache.spark.sql.classic.Dataset import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec, QueryStageExec, ShuffleQueryStageExec} @@ -91,6 +92,22 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt } } + test("inject analyzer rule - hidden column") { + withSession(Seq(_.injectResolutionRule(MyHiddenColumn))) { session: SparkSession => + val rel = LocalRelation( + AttributeReference("a", IntegerType)(), + AttributeReference("b", IntegerType)()) + rel.setTagValue[Long](LogicalPlan.PLAN_ID_TAG, 0L) + + val u = UnresolvedAttribute("x") + u.setTagValue[Long](LogicalPlan.PLAN_ID_TAG, 0L) + val proj = Project(Seq(u), rel) + + val df = Dataset.ofRows(session, proj) + assert(df.schema.fieldNames === Array("x")) + } + } + test("inject post hoc resolution analyzer rule") { withSession(Seq(_.injectPostHocResolutionRule(MyRule))) { session => assert(session.sessionState.analyzer.postHocResolutionRules.contains(MyRule(session))) @@ -608,6 +625,19 @@ case class MyRule(spark: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan } +case class MyHiddenColumn(spark: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case rel: LocalRelation if rel.output.size == 2 => + // rel.output.size == 2 for idempotence + val newRel = rel.copy( + output = rel.output :+ AttributeReference("x", IntegerType)() + ) + assert(rel.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(0L)) + newRel.setTagValue(LogicalPlan.PLAN_ID_TAG, 0L) + newRel + } +} + case class MyCheckRule(spark: SparkSession) extends (LogicalPlan => Unit) { override def apply(plan: LogicalPlan): Unit = { } } From a14a944dcbc11324460a8e614d4a0f064caf90d4 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 17 Dec 2025 20:12:16 -0800 Subject: [PATCH 305/400] [SPARK-54746][DOC] Fix wrong link in SQL Scripting page ### What changes were proposed in this pull request? Fix wrong link in SQL Scripting page image ### Why are the changes needed? Fix docs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually checked the new built docs, the hyperlink is correct now. https://github.com/pan3793/spark/actions/runs/20324237735/artifacts/4907052988 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53519 from pan3793/SPARK-54746. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 40879c6cd2e69019f37dadce5b323926f35b2831) Signed-off-by: Dongjoon Hyun --- docs/sql-ref-scripting.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-ref-scripting.md b/docs/sql-ref-scripting.md index e9407cff2301f..7d26bd07fed51 100644 --- a/docs/sql-ref-scripting.md +++ b/docs/sql-ref-scripting.md @@ -26,7 +26,7 @@ This is followed by the compound statement body, which consists of: - Flow control statements include loops over predicate expressions, [FOR](control-flow/for-stmt.html) loops over query results, conditional logic such as [IF](control-flow/if-stmt.html) and [CASE](control-flow/case-stmt.html), and means to break out loops such as [LEAVE](control-flow/leave-stmt.html) and [ITERATE](control-flow/iterate-stmt.html). - DDL statements such as `ALTER`, `CREATE`, `DROP`. -- DML statements [INSERT](sql-ref-syntax-dml-insert-into.html). +- DML statements [INSERT](sql-ref-syntax-dml-insert-table.html). - [Queries](sql-ref-syntax-qry-select.html) that return result sets to the invoker of the script. - [SET](sql-ref-syntax-aux-set-var.html) statements to set local variables as well as session variables. - The [EXECUTE IMMEDIATE](sql-ref-syntax-aux-exec-imm.html) statement. From aedd42be645e7e7492d240989d9325e750ddb843 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Thu, 18 Dec 2025 15:43:25 +0800 Subject: [PATCH 306/400] =?UTF-8?q?[SPARK-54621][SQL]=20Merge=20Into=20Upd?= =?UTF-8?q?ate=20Set=20*=20preserve=20nested=20fields=20if=20=E2=80=A6coer?= =?UTF-8?q?ceNestedTypes=20is=20enabled?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? The 'struct coercion' feature for MERGE INTO (allowing it to pass if assigning a struct with less fields into a struct with more fields) is turned off in a flag in https://github.com/apache/spark/pull/53229 due to some ambiguity in behavior, but was not removed because the community wanted to try it. We want to still keep it under a flag, but we make a choice about which behavior to support when the flag is on. In particular, we want UPDATE SET * to explode to all nested struct fields, so that in this scenario, existing nested struct fields are preserved. ### Why are the changes needed? aokolnychyi tested the feature and thinks that even if it is behind the experimental flag, we should take the stance for now that UPDATE SET * should explode to all nested fields vs top level columns. The rationale being: * its always safer to not override user values with null * Spark in general tries to treat nested fields like columns * there's already a way for the user to override the whole struct (and nullify non-existing fields) by specifying the struct explicitly, ie UPDATE SET struct = source.struct ### Does this PR introduce _any_ user-facing change? No, the whole feature is new and hidden behind an experimental flag. ### How was this patch tested? Existing tests (some output changes to not be null) ### Was this patch authored or co-authored using generative AI tooling? No Closes #53360 from szehon-ho/SPARK-54621. Authored-by: Szehon Ho Signed-off-by: Wenchen Fan (cherry picked from commit 92e5b3644d567392024464e4b3bd5f28fe69a550) Signed-off-by: Wenchen Fan --- .../catalyst/analysis/AssignmentUtils.scala | 163 ++- .../connector/MergeIntoTableSuiteBase.scala | 1111 ++++++++++++++--- 2 files changed, 1091 insertions(+), 183 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala index 6c7b0626e81e6..df4b0646ed42f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentUtils.scala @@ -19,15 +19,17 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.TableOutputResolver.DefaultValueFillMode.{NONE, RECURSE} -import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, Expression, GetStructField, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, CreateNamedStruct, Expression, GetStructField, If, IsNull, Literal} import org.apache.spark.sql.catalyst.plans.logical.Assignment import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getDefaultValueExprOrNullLit import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -72,7 +74,8 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { assignments, addError = err => errors += err, colPath = Seq(attr.name), - coerceNestedTypes) + coerceNestedTypes, + fromStar) } if (errors.nonEmpty) { @@ -156,7 +159,8 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { assignments: Seq[Assignment], addError: String => Unit, colPath: Seq[String], - coerceNestedTypes: Boolean = false): Expression = { + coerceNestedTypes: Boolean = false, + updateStar: Boolean = false): Expression = { val (exactAssignments, otherAssignments) = assignments.partition { assignment => assignment.key.semanticEquals(colExpr) @@ -178,11 +182,25 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { } else if (exactAssignments.isEmpty && fieldAssignments.isEmpty) { TableOutputResolver.checkNullability(colExpr, col, conf, colPath) } else if (exactAssignments.nonEmpty) { - val value = exactAssignments.head.value - val coerceMode = if (coerceNestedTypes) RECURSE else NONE - val resolvedValue = TableOutputResolver.resolveUpdate("", value, col, conf, addError, - colPath, coerceMode) - resolvedValue + if (updateStar && SQLConf.get.coerceMergeNestedTypes) { + val value = exactAssignments.head.value + col.dataType match { + case _: StructType => + // Expand assignments to leaf fields (fixNullExpansion is applied inside) + applyNestedFieldAssignments(col, colExpr, value, addError, colPath, + coerceNestedTypes) + case _ => + // For non-struct types, resolve directly + val coerceMode = if (coerceNestedTypes) RECURSE else NONE + TableOutputResolver.resolveUpdate("", value, col, conf, addError, colPath, + coerceMode) + } + } else { + val value = exactAssignments.head.value + val coerceMode = if (coerceNestedTypes) RECURSE else NONE + TableOutputResolver.resolveUpdate("", value, col, conf, addError, + colPath, coerceMode) + } } else { applyFieldAssignments(col, colExpr, fieldAssignments, addError, colPath, coerceNestedTypes) } @@ -211,7 +229,67 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { case otherType => addError( "Updating nested fields is only supported for StructType but " + - s"'${colPath.quoted}' is of type $otherType") + s"'${colPath.quoted}' is of type $otherType") + colExpr + } + } + + private def applyNestedFieldAssignments( + col: Attribute, + colExpr: Expression, + value: Expression, + addError: String => Unit, + colPath: Seq[String], + coerceNestedTypes: Boolean): Expression = { + + col.dataType match { + case structType: StructType => + val fieldAttrs = DataTypeUtils.toAttributes(structType) + + val updatedFieldExprs = fieldAttrs.zipWithIndex.map { case (fieldAttr, ordinal) => + val fieldPath = colPath :+ fieldAttr.name + val targetFieldExpr = GetStructField(colExpr, ordinal, Some(fieldAttr.name)) + + // Try to find a corresponding field in the source value by name + val sourceFieldValue: Expression = value.dataType match { + case valueStructType: StructType => + valueStructType.fields.find(f => conf.resolver(f.name, fieldAttr.name)) match { + case Some(matchingField) => + // Found matching field in source, extract it + val fieldIndex = valueStructType.fieldIndex(matchingField.name) + GetStructField(value, fieldIndex, Some(matchingField.name)) + case None => + // Field doesn't exist in source, use target's current value with null check + TableOutputResolver.checkNullability(targetFieldExpr, fieldAttr, conf, fieldPath) + } + case _ => + // Value is not a struct, cannot extract field + addError(s"Cannot assign non-struct value to struct field '${fieldPath.quoted}'") + Literal(null, fieldAttr.dataType) + } + + // Recurse or resolve based on field type + fieldAttr.dataType match { + case _: StructType => + // Field is a struct, recurse + applyNestedFieldAssignments(fieldAttr, targetFieldExpr, + sourceFieldValue, addError, fieldPath, coerceNestedTypes) + case _ => + // Field is not a struct, resolve with TableOutputResolver + val coerceMode = if (coerceNestedTypes) RECURSE else NONE + TableOutputResolver.resolveUpdate("", sourceFieldValue, fieldAttr, conf, addError, + fieldPath, coerceMode) + } + } + val namedStruct = toNamedStruct(structType, updatedFieldExprs) + + // Prevent unnecessary null struct expansion + fixNullExpansion(colExpr, value, structType, namedStruct, colPath) + + case otherType => + addError( + "Updating nested fields is only supported for StructType but " + + s"'${colPath.quoted}' is of type $otherType") colExpr } } @@ -223,6 +301,73 @@ object AssignmentUtils extends SQLConfHelper with CastSupport { CreateNamedStruct(namedStructExprs) } + /** + * Checks if target struct has extra fields compared to source struct, recursively. + */ + private def hasExtraTargetFields(targetType: StructType, sourceType: DataType): Boolean = { + sourceType match { + case sourceStructType: StructType => + targetType.fields.exists { targetField => + sourceStructType.fields.find(f => conf.resolver(f.name, targetField.name)) match { + case Some(sourceField) => + // Check nested structs recursively + (targetField.dataType, sourceField.dataType) match { + case (targetNested: StructType, sourceNested) => + hasExtraTargetFields(targetNested, sourceNested) + case _ => false + } + case None => true // target has extra field not in source + } + } + case _ => + // Should be caught earlier + throw SparkException.internalError( + s"Source type must be StructType but found: $sourceType") + } + } + + /** + * As UPDATE SET * assigns struct fields individually (preserving existing fields), + * this will lead to indiscriminate null expansion, ie, a struct is created where all + * fields are null. Wraps a struct assignment with a condition to return null + * if both conditions are true: + * + * - source struct is null + * - target struct is null OR target struct is same as source struct + * + * If the condition is not true, we preserve the original structure. + * This includes cases where the source was a struct of nulls, + * or there were any extra target fields (including null ones), + * both cases retain the assignment to a struct of nulls. + * + * @param key the original assignment key (target struct) expression + * @param value the original assignment value (source struct) expression + * @param structType the target struct type + * @param structExpression the result create struct expression result to wrap + * @param colPath the column path for error reporting + * @return the wrapped expression with null checks + */ + private def fixNullExpansion( + key: Expression, + value: Expression, + structType: StructType, + structExpression: Expression, + colPath: Seq[String]): Expression = { + if (key.nullable) { + val condition = if (hasExtraTargetFields(structType, value.dataType)) { + // extra target fields: return null iff source struct is null and target struct is null + And(IsNull(value), IsNull(key)) + } else { + // schemas match: return null iff source struct is null + IsNull(value) + } + + If(condition, Literal(null, structExpression.dataType), structExpression) + } else { + structExpression + } + } + /** * Checks whether assignments are aligned and compatible with table columns. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index 5d1173b5a1a54..ea19a916b7cc8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -3240,9 +3240,8 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase checkAnswer( sql(s"SELECT * FROM $tableNameAsString"), Seq( - Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "sales"), + Row(1, Row(10, Row(Seq(1, 2), Map("c" -> "d"), false)), "sales"), Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) - } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { sql(mergeStmt) @@ -4776,8 +4775,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - - test("merge with null struct - update field") { + test("merge with struct of nulls") { withTempView("source") { createAndInitTable( s"""pk INT NOT NULL, @@ -4797,9 +4795,10 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase StructField("dep", StringType) )) + // Source has a struct with null field values (not a null struct) val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") + Row(1, Row(null, null), "engineering"), + Row(2, Row(null, null), "finance") ) spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") @@ -4808,31 +4807,32 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase s"""MERGE INTO $tableNameAsString t USING source |ON t.pk = source.pk |WHEN MATCHED THEN - | UPDATE SET s = source.s + | UPDATE SET * |WHEN NOT MATCHED THEN | INSERT * |""".stripMargin) + // Struct of null values should be preserved, not converted to null struct checkAnswer( sql(s"SELECT * FROM $tableNameAsString"), Seq( Row(0, Row(1, "a"), "sales"), - Row(1, null, "hr"), - Row(2, null, "finance"))) + Row(1, Row(null, null), "engineering"), + Row(2, Row(null, null), "finance"))) } sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - test("merge with null struct into non-nullable struct column") { + test("merge with null struct into struct of nulls") { withTempView("source") { createAndInitTable( s"""pk INT NOT NULL, - |s STRUCT NOT NULL, + |s STRUCT, |dep STRING""".stripMargin, """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + |{ "pk": 1, "s": { "c1": null, "c2": null }, "dep": "hr" }""" .stripMargin) - // Source table has null for the struct column + // Source table matches target table schema val sourceTableSchema = StructType(Seq( StructField("pk", IntegerType), StructField("s", StructType(Seq( @@ -4842,61 +4842,60 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase StructField("dep", StringType) )) + // Source has a null struct (not a struct of nulls) val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") + Row(1, null, "engineering") ) spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") - // Should throw an exception when trying to insert/update null into NOT NULL column - val exception = intercept[Exception] { - sql( - s"""MERGE INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin) - } - assert(exception.getMessage.contains( - "NULL value appeared in non-nullable field")) + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + // Null struct should override struct of nulls + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a"), "sales"), + Row(1, null, "engineering"))) } sql(s"DROP TABLE IF EXISTS $tableNameAsString") } - test("merge with with null struct with missing nested field") { + test("merge with null struct into struct of nulls with extra target field") { Seq(true, false).foreach { withSchemaEvolution => Seq(true, false).foreach { coerceNestedTypes => withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> coerceNestedTypes.toString) { withTempView("source") { - // Target table has nested struct with fields c1 and c2 + // Target has struct with 3 fields, row 1 has all nulls including extra field c3 createAndInitTable( s"""pk INT NOT NULL, - |s STRUCT>, + |s STRUCT, |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": 10 }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": null, "c2": null, "c3": null }, "dep": "hr" }""" .stripMargin) - // Source table has null for the nested struct + // Source table has struct with 2 fields (missing c3) val sourceTableSchema = StructType(Seq( StructField("pk", IntegerType), StructField("s", StructType(Seq( StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType) - // missing field 'b' - ))) + StructField("c2", StringType) + // missing field c3 ))), StructField("dep", StringType) )) + // Source has a null struct (not a struct of nulls) val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") + Row(1, null, "engineering") ) spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") @@ -4913,12 +4912,12 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase if (coerceNestedTypes && withSchemaEvolution) { sql(mergeStmt) + // Because target has extra field c3, we preserve struct of nulls checkAnswer( sql(s"SELECT * FROM $tableNameAsString"), Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, null, "engineering"), - Row(2, null, "finance"))) + Row(0, Row(1, "a", 10), "sales"), + Row(1, Row(null, null, null), "engineering"))) } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { sql(mergeStmt) @@ -4933,114 +4932,42 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase } } - test("merge null struct with schema evolution - source with missing and extra nested fields") { - Seq(true, false).foreach { withSchemaEvolution => - Seq(true, false).foreach { coerceNestedTypes => - withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> - coerceNestedTypes.toString) { - withTempView("source") { - // Target table has nested struct with fields c1 and c2 - createAndInitTable( - s"""pk INT NOT NULL, - |s STRUCT>, - |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" - .stripMargin) - - // Source table has missing field 'b' and extra field 'c' in nested struct - val sourceTableSchema = StructType(Seq( - StructField("pk", IntegerType), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType), - // missing field 'b' - StructField("c", StringType) // extra field 'c' - ))) - ))), - StructField("dep", StringType) - )) - - val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") - ) - spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) - .createOrReplaceTempView("source") - - val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" - val mergeStmt = - s"""MERGE $schemaEvolutionClause - |INTO $tableNameAsString t USING source - |ON t.pk = source.pk - |WHEN MATCHED THEN - | UPDATE SET * - |WHEN NOT MATCHED THEN - | INSERT * - |""".stripMargin - - if (coerceNestedTypes && withSchemaEvolution) { - // extra nested field is added - sql(mergeStmt) - checkAnswer( - sql(s"SELECT * FROM $tableNameAsString"), - Seq( - Row(0, Row(1, Row(10, "x", null)), "sales"), - Row(1, null, "engineering"), - Row(2, null, "finance"))) - } else { - val exception = intercept[org.apache.spark.sql.AnalysisException] { - sql(mergeStmt) - } - assert(exception.errorClass.get == - "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") - } - } - } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") - } - } - } - - test("merge null struct with non-nullable nested field - source with missing " + - "and extra nested fields") { + test("merge with struct of nulls with missing source field") { Seq(true, false).foreach { withSchemaEvolution => Seq(true, false).foreach { coerceNestedTypes => withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> coerceNestedTypes.toString) { withTempView("source") { + // Target has struct with 3 fields createAndInitTable( s"""pk INT NOT NULL, - |s STRUCT>, + |s STRUCT, |dep STRING""".stripMargin, - """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } - |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": 10 }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b", "c3": 20 }, "dep": "hr" }""" .stripMargin) + // Source table has struct with 2 fields (missing field c3) val sourceTableSchema = StructType(Seq( StructField("pk", IntegerType), StructField("s", StructType(Seq( StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType), - StructField("c", StringType) - ))) + StructField("c2", StringType) + // missing field c3 ))), StructField("dep", StringType) )) + // Source has a struct with two null field values (not a null struct) val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") + Row(1, Row(null, null), "engineering") ) spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" val mergeStmt = - s"""MERGE $schemaEvolutionClause - |INTO $tableNameAsString t USING source + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source |ON t.pk = source.pk |WHEN MATCHED THEN | UPDATE SET * @@ -5048,67 +4975,57 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase | INSERT * |""".stripMargin - val exception = intercept[org.apache.spark.sql.AnalysisException] { + if (coerceNestedTypes && withSchemaEvolution) { sql(mergeStmt) + // Struct of null values should be preserved, not converted to null struct + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a", 10), "sales"), + Row(1, Row(null, null, 20), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") } - assert(exception.errorClass.get == - "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") - assert(exception.getMessage.contains( - "Cannot find data for the output column `s`.`c2`.`b`")) } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } } - test("merge with null struct using default value") { + test("merge with struct of nulls with missing source field and null target field") { Seq(true, false).foreach { withSchemaEvolution => Seq(true, false).foreach { coerceNestedTypes => withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> coerceNestedTypes.toString) { withTempView("source") { - sql( - s"""CREATE TABLE $tableNameAsString ( - | pk INT NOT NULL, - | s STRUCT> DEFAULT - | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), - | dep STRING) - |PARTITIONED BY (dep) - |""".stripMargin) - - val initialSchema = StructType(Seq( - StructField("pk", IntegerType, nullable = false), - StructField("s", StructType(Seq( - StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType), - StructField("b", StringType) - ))) - ))), - StructField("dep", StringType) - )) - val initialData = Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, Row(2, Row(20, "y")), "hr") - ) - spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) - .writeTo(tableNameAsString).append() + // Target has struct with 3 fields, but row 1 has null for the extra field c3 + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a", "c3": 10 }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b", "c3": null }, "dep": "hr" }""" + .stripMargin) + // Source table has struct with 2 fields (missing field c3) val sourceTableSchema = StructType(Seq( StructField("pk", IntegerType), StructField("s", StructType(Seq( StructField("c1", IntegerType), - StructField("c2", StructType(Seq( - StructField("a", IntegerType) - ))) + StructField("c2", StringType) + // missing field c3 ))), StructField("dep", StringType) )) + // Source has a struct with two null field values (not a null struct) val data = Seq( - Row(1, null, "engineering"), - Row(2, null, "finance") + Row(1, Row(null, null), "engineering") ) spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) .createOrReplaceTempView("source") @@ -5125,25 +5042,871 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase if (coerceNestedTypes && withSchemaEvolution) { sql(mergeStmt) + // Struct of null values should be preserved, not converted to null struct checkAnswer( sql(s"SELECT * FROM $tableNameAsString"), Seq( - Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, null, "engineering"), - Row(2, null, "finance"))) + Row(0, Row(1, "a", 10), "sales"), + Row(1, Row(null, null, null), "engineering"))) } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { sql(mergeStmt) } - assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") } } - sql(s"DROP TABLE IF EXISTS $tableNameAsString") } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") } } } + test("merge with null struct - update field") { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) + + // Source table matches target table schema + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET s = source.s + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, "a"), "sales"), + Row(1, null, "hr"), + Row(2, null, "finance"))) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge with null nested struct in doubly nested struct") { + Seq(true, false).foreach { withSchemaEvolution => + + withTempView("source") { + // Target has doubly nested struct with 2 fields in innermost struct + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "foo" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "bar" } }, "dep": "hr" }""" + .stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + + // Source has a row where the nested struct (c2) is null + val data = Seq( + Row(1, Row(3, null), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + sql(mergeStmt) + // Null nested struct should be preserved + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "foo")), "sales"), + Row(1, Row(3, null), "engineering"))) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + + + test("merge with null struct into non-nullable struct column") { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT NOT NULL, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": "a" }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": "b" }, "dep": "hr" }""" + .stripMargin) + + // Source table has null for the struct column + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StringType) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + // Should throw an exception when trying to insert/update null into NOT NULL column + val exception = intercept[Exception] { + sql( + s"""MERGE INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + } + assert(exception.getMessage.contains( + "NULL value appeared in non-nullable field")) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + + test("merge with with null struct with missing nested field") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target table has nested struct with fields c1 and c2 + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + .stripMargin) + + // Source table has null for the nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(null, Row(null, "y")), "engineering"), + Row(2, null, "finance"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge with null source struct with extra target source field being null") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target table has nested struct, row 1 has null for field 'b' (missing in source) + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": null } }, "dep": "hr" }""" + .stripMargin) + + // Source table has struct with missing nested field 'b' + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + // missing field 'b' + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + // It's not immediately obvious, but because the target had extra fields + // we preserve them despite them being null (and thus retain the struct of nulls) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(null, Row(null, null)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge with null source struct with extra target field in doubly nested struct") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target has struct nested in struct, with extra field 'y' in innermost struct + val targetTableSchema = StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StructType(Seq( + StructField("x", IntegerType), + StructField("y", StringType) + ))) + ))) + )) + + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", targetTableSchema), + Column.create("dep", StringType)) + createTable(columns) + + val targetData = Seq( + Row(0, Row(1, Row(10, Row(100, "foo"))), "sales"), + Row(1, Row(2, Row(20, Row(200, null))), "hr") + ) + val targetDataSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", targetTableSchema), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetDataSchema) + .writeTo(tableNameAsString).append() + + // Source has struct with missing field 'y' in innermost struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StructType(Seq( + StructField("x", IntegerType) + // missing field 'y' + ))) + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + // Because the target had extra field 'y' which is null, + // we preserve it and retain the struct of nulls + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, Row(100, "foo"))), "sales"), + Row(1, Row(null, Row(null, Row(null, null))), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge with null source and target nested struct with extra target field") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target has struct nested in struct, with extra field 'y' in innermost struct + val targetTableSchema = StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StructType(Seq( + StructField("x", IntegerType), + StructField("y", StringType) + ))) + ))) + )) + + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", targetTableSchema), + Column.create("dep", StringType)) + createTable(columns) + + // Target data has null for innermost struct 'b' which has the extra field 'y' + val targetData = Seq( + Row(0, Row(1, Row(10, Row(100, "foo"))), "sales"), + Row(1, Row(2, Row(20, null)), "hr") + ) + val targetDataSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", targetTableSchema), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetDataSchema) + .writeTo(tableNameAsString).append() + + // Source has struct with missing field 'y' in innermost struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StructType(Seq( + StructField("x", IntegerType) + // missing field 'y' + ))) + ))) + ))), + StructField("dep", StringType) + )) + + // Source data also has null for innermost struct 'b' + val data = Seq( + Row(1, Row(3, Row(30, null)), "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + // Both source and target have null for 'b', which should remain null + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, Row(100, "foo"))), "sales"), + Row(1, Row(3, Row(30, null)), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge with null source struct with extra target field in struct inside array") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target has struct with array of structs, with extra field 'y' in array element struct + val arrayElementSchema = StructType(Seq( + StructField("x", IntegerType), + StructField("y", StringType) + )) + val targetTableSchema = StructType(Seq( + StructField("c1", IntegerType), + StructField("arr", ArrayType(arrayElementSchema)) + )) + + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", targetTableSchema), + Column.create("dep", StringType)) + createTable(columns) + + val targetData = Seq( + Row(0, Row(1, Seq(Row(100, "foo"), Row(101, "bar"))), "sales"), + Row(1, Row(2, Seq(Row(200, null), Row(201, null))), "hr") + ) + val targetDataSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", targetTableSchema), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetDataSchema) + .writeTo(tableNameAsString).append() + + // Source has struct with missing field 'y' in array element struct + val sourceArrayElementSchema = StructType(Seq( + StructField("x", IntegerType) + // missing field 'y' + )) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("arr", ArrayType(sourceArrayElementSchema)) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + // Because the target had extra field 'y' which is within an array, + // it cannot be referenced and so we do not preserve it and allow source null + // to override it. + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Seq(Row(100, "foo"), Row(101, "bar"))), "sales"), + Row(1, null, "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge with null source struct with extra null target field in struct containing array") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + val arrayElementSchema = StructType(Seq( + StructField("x", IntegerType) + )) + val targetTableSchema = StructType(Seq( + StructField("c1", IntegerType), + StructField("arr", ArrayType(arrayElementSchema)), + StructField("c2", StringType) // extra field at nested struct level + )) + + val columns = Array( + Column.create("pk", IntegerType, false), + Column.create("s", targetTableSchema), + Column.create("dep", StringType)) + createTable(columns) + + val targetData = Seq( + Row(0, Row(1, Seq(Row(100), Row(101)), "foo"), "sales"), + Row(1, Row(2, Seq(Row(200), Row(201)), null), "hr") // c2 is null + ) + val targetDataSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", targetTableSchema), + StructField("dep", StringType) + )) + spark.createDataFrame(spark.sparkContext.parallelize(targetData), targetDataSchema) + .writeTo(tableNameAsString).append() + + // Source has struct missing field 'c2' + val sourceArrayElementSchema = StructType(Seq( + StructField("x", IntegerType) + )) + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("arr", ArrayType(sourceArrayElementSchema)) + // missing field 'c2' + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + // Because the target had extra field 'c2' which is null, + // we preserve it and retain the struct of nulls + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Seq(Row(100), Row(101)), "foo"), "sales"), + Row(1, Row(null, null, null), "engineering"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge null struct with schema evolution - source with missing and extra nested fields") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + // Target table has nested struct with fields c1 and c2 + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + .stripMargin) + + // Source table has missing field 'b' and extra field 'c' in nested struct + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + // missing field 'b' + StructField("c", StringType) // extra field 'c' + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + // extra nested field is added + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x", null)), "sales"), + Row(1, Row(null, Row(null, "y", null)), "engineering"), + Row(2, null, "finance"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + + test("merge null struct with non-nullable nested field - source with missing " + + "and extra nested fields") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + createAndInitTable( + s"""pk INT NOT NULL, + |s STRUCT>, + |dep STRING""".stripMargin, + """{ "pk": 0, "s": { "c1": 1, "c2": { "a": 10, "b": "x" } }, "dep": "sales" } + |{ "pk": 1, "s": { "c1": 2, "c2": { "a": 20, "b": "y" } }, "dep": "hr" }""" + .stripMargin) + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("c", StringType) + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause + |INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == + "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + assert(exception.getMessage.contains( + "Cannot find data for the output column `s`.`c2`.`b`")) + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + } + + test("merge with null struct using default value") { + Seq(true, false).foreach { withSchemaEvolution => + Seq(true, false).foreach { coerceNestedTypes => + withSQLConf(SQLConf.MERGE_INTO_NESTED_TYPE_COERCION_ENABLED.key -> + coerceNestedTypes.toString) { + withTempView("source") { + sql( + s"""CREATE TABLE $tableNameAsString ( + | pk INT NOT NULL, + | s STRUCT> DEFAULT + | named_struct('c1', 999, 'c2', named_struct('a', 999, 'b', 'default')), + | dep STRING) + |PARTITIONED BY (dep) + |""".stripMargin) + + val initialSchema = StructType(Seq( + StructField("pk", IntegerType, nullable = false), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType) + ))) + ))), + StructField("dep", StringType) + )) + val initialData = Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(2, Row(20, "y")), "hr") + ) + spark.createDataFrame(spark.sparkContext.parallelize(initialData), initialSchema) + .writeTo(tableNameAsString).append() + + val sourceTableSchema = StructType(Seq( + StructField("pk", IntegerType), + StructField("s", StructType(Seq( + StructField("c1", IntegerType), + StructField("c2", StructType(Seq( + StructField("a", IntegerType) + ))) + ))), + StructField("dep", StringType) + )) + + val data = Seq( + Row(1, null, "engineering"), + Row(2, null, "finance") + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), sourceTableSchema) + .createOrReplaceTempView("source") + + val schemaEvolutionClause = if (withSchemaEvolution) "WITH SCHEMA EVOLUTION" else "" + val mergeStmt = + s"""MERGE $schemaEvolutionClause INTO $tableNameAsString t USING source + |ON t.pk = source.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + + if (coerceNestedTypes && withSchemaEvolution) { + sql(mergeStmt) + checkAnswer( + sql(s"SELECT * FROM $tableNameAsString"), + Seq( + Row(0, Row(1, Row(10, "x")), "sales"), + Row(1, Row(null, Row(null, "y")), "engineering"), + Row(2, null, "finance"))) + } else { + val exception = intercept[org.apache.spark.sql.AnalysisException] { + sql(mergeStmt) + } + assert(exception.errorClass.get == "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA") + } + } + sql(s"DROP TABLE IF EXISTS $tableNameAsString") + } + } + } + } test("merge with source missing struct column with default value") { withTempView("source") { @@ -5258,8 +6021,8 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase checkAnswer( sql(s"SELECT * FROM $tableNameAsString"), Seq( - Row(1, Row(10, Row(20, null)), "sales"), - Row(2, Row(20, Row(30, null)), "engineering"))) + Row(1, Row(10, Row(20, true)), "sales"), + Row(2, Row(20, Row(30, false)), "engineering"))) } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { sql(mergeStmt) @@ -5918,7 +6681,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase checkAnswer( sql(s"SELECT * FROM $tableNameAsString"), Seq( - Row(1, Row(10, Row(null, Map("c" -> "d"), false)), "sales"), + Row(1, Row(10, Row(Seq(1, 2), Map("c" -> "d"), false)), "sales"), Row(2, Row(20, Row(null, Map("e" -> "f"), true)), "engineering"))) } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { @@ -6267,7 +7030,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"SELECT * FROM $tableNameAsString"), Seq( Row(0, Row(1, Row(10, "x")), "sales"), - Row(1, null, "engineering"), + Row(1, Row(null, Row(null, "y")), "engineering"), Row(2, null, "finance"))) } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { @@ -6371,7 +7134,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase sql(s"SELECT * FROM $tableNameAsString"), Seq( Row(0, Row(1, Row(10, "x", null)), "sales"), - Row(1, null, "engineering"), + Row(1, Row(null, Row(null, "y", null)), "engineering"), Row(2, null, "finance"))) } else { val exception = intercept[org.apache.spark.sql.AnalysisException] { From 819a8cca6a1db9722a64aa457ba263ef495bd6e7 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Thu, 18 Dec 2025 15:48:32 +0800 Subject: [PATCH 307/400] [SPARK-46741][SQL] Cache Table with CTE won't work MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Reopen https://github.com/apache/spark/pull/44767 Cache Table with CTE won't work, there are two reasons 1. In the current code CTE in CacheTableAsSelect will be inlined 2. CTERelation Ref and Def didn't handle the CTEId doCanonicalize issue Cause the current case can't be matched. ### Why are the changes needed? Fix Bug ### Does this PR introduce _any_ user-facing change? Yea, Cache table with CTE can work after this pr For added `cache.sql` final query `EXPLAIN EXTENDED SELECT * FROM cache_nested_cte_table;` Before this pr, the plan as below, cache won't work. 截屏2025-12-05 11 22 05 After this pr 截屏2025-12-05 11 32 38 ### How was this patch tested? Added UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #53333 from AngersZhuuuu/SPARK-46741. Authored-by: Angerszhuuuu Signed-off-by: Wenchen Fan (cherry picked from commit 8f696791507d85a684d2e12f579c81e454b87234) Signed-off-by: Wenchen Fan --- .../catalyst/normalizer/NormalizeCTEIds.scala | 54 +++++++++++++++++++ .../catalyst/plans/logical/v2Commands.scala | 7 ++- .../internal/BaseSessionStateBuilder.scala | 2 + .../apache/spark/sql/CachedTableSuite.scala | 38 +++++++++++++ 4 files changed, 100 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala new file mode 100644 index 0000000000000..1b1b526e78140 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.normalizer + +import org.apache.spark.sql.catalyst.plans.logical.{CacheTableAsSelect, CTERelationRef, LogicalPlan, UnionLoop, UnionLoopRef, WithCTE} +import org.apache.spark.sql.catalyst.rules.Rule + +object NormalizeCTEIds extends Rule[LogicalPlan]{ + override def apply(plan: LogicalPlan): LogicalPlan = { + val curId = new java.util.concurrent.atomic.AtomicLong() + plan transformDown { + + case ctas @ CacheTableAsSelect(_, plan, _, _, _, _, _) => + ctas.copy(plan = apply(plan)) + + case withCTE @ WithCTE(plan, cteDefs) => + val defIdToNewId = withCTE.cteDefs.map(_.id).map((_, curId.getAndIncrement())).toMap + val normalizedPlan = canonicalizeCTE(plan, defIdToNewId) + val newCteDefs = cteDefs.map { cteDef => + val normalizedCteDef = canonicalizeCTE(cteDef.child, defIdToNewId) + cteDef.copy(child = normalizedCteDef, id = defIdToNewId(cteDef.id)) + } + withCTE.copy(plan = normalizedPlan, cteDefs = newCteDefs) + } + } + + def canonicalizeCTE(plan: LogicalPlan, defIdToNewId: Map[Long, Long]): LogicalPlan = { + plan.transformDownWithSubqueries { + // For nested WithCTE, if defIndex didn't contain the cteId, + // means it's not current WithCTE's ref. + case ref: CTERelationRef if defIdToNewId.contains(ref.cteId) => + ref.copy(cteId = defIdToNewId(ref.cteId)) + case unionLoop: UnionLoop if defIdToNewId.contains(unionLoop.id) => + unionLoop.copy(id = defIdToNewId(unionLoop.id)) + case unionLoopRef: UnionLoopRef if defIdToNewId.contains(unionLoopRef.loopId) => + unionLoopRef.copy(loopId = defIdToNewId(unionLoopRef.loopId)) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 72274ee9bf174..fab64d771093f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1742,7 +1742,8 @@ case class CacheTableAsSelect( isLazy: Boolean, options: Map[String, String], isAnalyzed: Boolean = false, - referredTempFunctions: Seq[String] = Seq.empty) extends AnalysisOnlyCommand { + referredTempFunctions: Seq[String] = Seq.empty) + extends AnalysisOnlyCommand with CTEInChildren { override protected def withNewChildrenInternal( newChildren: IndexedSeq[LogicalPlan]): CacheTableAsSelect = { assert(!isAnalyzed) @@ -1757,6 +1758,10 @@ case class CacheTableAsSelect( // Collect the referred temporary functions from AnalysisContext referredTempFunctions = ac.referredTempFunctionNames.toSeq) } + + override def withCTEDefs(cteDefs: Seq[CTERelationDef]): LogicalPlan = { + copy(plan = WithCTE(plan, cteDefs)) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index ef829eaae68c9..a8c23c8e126f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTr import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} +import org.apache.spark.sql.catalyst.normalizer.NormalizeCTEIds import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -403,6 +404,7 @@ abstract class BaseSessionStateBuilder( } protected def planNormalizationRules: Seq[Rule[LogicalPlan]] = { + NormalizeCTEIds +: extensions.buildPlanNormalizationRules(session) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 12d26c4e195f1..880d8d72c73e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -2598,6 +2598,44 @@ class CachedTableSuite extends QueryTest with SQLTestUtils } } + test("SPARK-46741: Cache Table with CTE should work") { + withTempView("t1", "t2") { + sql( + """ + |CREATE TEMPORARY VIEW t1 + |AS + |SELECT * FROM VALUES (0, 0), (1, 1), (2, 2) AS t(c1, c2) + |""".stripMargin) + sql( + """ + |CREATE TEMPORARY VIEW t2 AS + |WITH v as ( + | SELECT c1 + c1 c3 FROM t1 + |) + |SELECT SUM(c3) s FROM v + |""".stripMargin) + sql( + """ + |CACHE TABLE cache_nested_cte_table + |WITH + |v AS ( + | SELECT c1 * c2 c3 from t1 + |) + |SELECT SUM(c3) FROM v + |EXCEPT + |SELECT s FROM t2 + |""".stripMargin) + + val df = sql("SELECT * FROM cache_nested_cte_table") + + val inMemoryTableScan = collect(df.queryExecution.executedPlan) { + case i: InMemoryTableScanExec => i + } + assert(inMemoryTableScan.size == 1) + checkAnswer(df, Row(5) :: Nil) + } + } + private def cacheManager = spark.sharedState.cacheManager private def pinTable( From 1d6665ae8e7d6caac75e2b3ebb6588b4788f2727 Mon Sep 17 00:00:00 2001 From: Fu Chen Date: Thu, 18 Dec 2025 21:07:33 +0800 Subject: [PATCH 308/400] [SPARK-54749][SQL] Fix incorrect numOutputRows metric in OneRowRelationExec MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR moves the `numOutputRows` metric update logic from the val rdd initialization block to the `doExecute()` method in `OneRowRelationExec`. ### Why are the changes needed? Currently, the `numOutputRows` metric in `OneRowRelationExec` is incorrectly incremented twice in the codegen codebase (displaying 2 instead of 1 for a single row). before this PR: 企业微信截图_70a269d7-7e34-40a4-a0f4-abeae7283a79 after this PR: 企业微信截图_d7fab096-024e-440b-94e1-707296258c55 ### Does this PR introduce _any_ user-facing change? No, only bug fix. ### How was this patch tested? Added UT. ### Was this patch authored or co-authored using generative AI tooling? Closes #53520 from cfmcgrady/fix-onerowrelation-metrics. Authored-by: Fu Chen Signed-off-by: Wenchen Fan (cherry picked from commit a26c20953bb3262ce4da1f609457d7525e37af1a) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/ExistingRDD.scala | 14 ++++++++------ .../execution/metric/SQLMetricsSuite.scala | 19 +++++++++++++++++++ 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 6148fb30783e8..06085497de19a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -360,23 +360,25 @@ case class OneRowRelationExec() extends LeafExecNode override val output: Seq[Attribute] = Nil private val rdd: RDD[InternalRow] = { - val numOutputRows = longMetric("numOutputRows") session .sparkContext .parallelize(Seq(""), 1) .mapPartitionsInternal { _ => val proj = UnsafeProjection.create(Seq.empty[Expression]) - Iterator(proj.apply(InternalRow.empty)).map { r => - numOutputRows += 1 - r - } + Iterator(proj.apply(InternalRow.empty)) } } override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - protected override def doExecute(): RDD[InternalRow] = rdd + protected override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + rdd.map { r => + numOutputRows += 1 + r + } + } override def simpleString(maxFields: Int): String = s"$nodeName[]" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 402365a59eceb..f2e9121d566c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -107,6 +107,25 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils } } + test("SPARK-54749: OneRowRelation metrics") { + Seq((1L, "false"), (2L, "true")).foreach { case (nodeId, enableWholeStage) => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> enableWholeStage) { + val df = spark.sql("select 1 as c1") + val oneRowRelation = df.queryExecution.executedPlan.collect { + case oneRowRelation: OneRowRelationExec => oneRowRelation + } + df.collect() + sparkContext.listenerBus.waitUntilEmpty() + assert(oneRowRelation.size == 1) + + val expected = Map("number of output rows" -> 1L) + testSparkPlanMetrics(df.toDF(), 1, Map( + nodeId -> (("Scan OneRowRelation", expected)))) + } + } + } + + test("Recursive CTEs metrics") { withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { val df = sql( From 69aa912d9cdda3b9892e762751d6575b7491ffea Mon Sep 17 00:00:00 2001 From: Aleksandr Chernousov Date: Fri, 19 Dec 2025 11:43:10 +0800 Subject: [PATCH 309/400] [SPARK-54652][SQL] Complete conversion of IDENTIFIER() ### What changes were proposed in this pull request? I replace incorrect usages of "ctx.getText()" with "getIdentifierText(ctx)" ### Why are the changes needed? It blocks from using IDENTIFIER() in the remaining expected places ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Added new tests in identifier-clause.sql ### Was this patch authored or co-authored using generative AI tooling? Claude Sonnet 4.5 Closes #53407 from aleksandr-chernousov-db/SPARK-54652. Lead-authored-by: Aleksandr Chernousov Co-authored-by: Ubuntu Signed-off-by: Wenchen Fan (cherry picked from commit 1d8c11120fdee04fe88b536bf4cf11d1da5141eb) Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 10 +- .../catalyst/parser/DataTypeAstBuilder.scala | 2 +- .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 46 +-- .../spark/sql/execution/SparkSqlParser.scala | 12 +- .../identifier-clause-legacy.sql.out | 41 +++ .../identifier-clause.sql.out | 46 ++- .../sql-tests/inputs/identifier-clause.sql | 6 + .../results/identifier-clause-legacy.sql.out | 48 +++ .../results/identifier-clause.sql.out | 57 ++-- .../command/IdentifierClauseParserSuite.scala | 282 ++++++++++++++++++ 11 files changed, 483 insertions(+), 69 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/IdentifierClauseParserSuite.scala diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 9d942bc601592..02b7ec195a933 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -232,9 +232,9 @@ statement createTableClauses (AS? query)? #replaceTable | ANALYZE TABLE identifierReference partitionSpec? COMPUTE STATISTICS - (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze + (simpleIdentifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze | ANALYZE TABLES ((FROM | IN) identifierReference)? COMPUTE STATISTICS - (identifier)? #analyzeTables + (simpleIdentifier)? #analyzeTables | ALTER TABLE identifierReference ADD (COLUMN | COLUMNS) columns=qualifiedColTypeWithPositionList #addTableColumns @@ -356,7 +356,7 @@ statement | TRUNCATE TABLE identifierReference partitionSpec? #truncateTable | (MSCK)? REPAIR TABLE identifierReference (option=(ADD|DROP|SYNC) PARTITIONS)? #repairTable - | op=(ADD | LIST) identifier .*? #manageResource + | op=(ADD | LIST) simpleIdentifier .*? #manageResource | CREATE INDEX (IF errorCapturingNot EXISTS)? identifier ON TABLE? identifierReference (USING indexType=identifier)? LEFT_PAREN columns=multipartIdentifierPropertyList RIGHT_PAREN @@ -660,7 +660,7 @@ createFileFormat fileFormat : INPUTFORMAT inFmt=stringLit OUTPUTFORMAT outFmt=stringLit #tableFileFormat - | identifier #genericFileFormat + | simpleIdentifier #genericFileFormat ; storageHandler @@ -668,7 +668,7 @@ storageHandler ; resource - : identifier stringLit + : simpleIdentifier stringLit ; dmlStatementNoWith diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index 212c80a3cb435..51c846f93c1ec 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -497,7 +497,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with DataTypeE override def visitComplexColType(ctx: ComplexColTypeContext): StructField = withOrigin(ctx) { import ctx._ val structField = StructField( - name = errorCapturingIdentifier.getText, + name = getIdentifierText(errorCapturingIdentifier), dataType = typedVisit(dataType()), nullable = NULL == null) Option(commentSpec).map(visitCommentSpec).map(structField.withComment).getOrElse(structField) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 553161ea2db0a..ea565aeb7febf 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -461,7 +461,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { ctx) } - def computeStatisticsNotExpectedError(ctx: IdentifierContext): Throwable = { + def computeStatisticsNotExpectedError(ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", messageParameters = Map("ctx" -> toSQLStmt(ctx.getText)), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d9202f69fcea6..9adc755730f59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1235,7 +1235,7 @@ class AstBuilder extends DataTypeAstBuilder if (pVal.DEFAULT != null) { throw QueryParsingErrors.defaultColumnReferencesNotAllowedInPartitionSpec(ctx) } - val name = pVal.identifier.getText + val name = getIdentifierText(pVal.identifier) val value = Option(pVal.constant).map(v => { visitStringConstant(v, legacyNullAsString, keepPartitionSpecAsString) }) @@ -1958,11 +1958,11 @@ class AstBuilder extends DataTypeAstBuilder .flatMap(_.namedExpression.asScala) .map(typedVisit[Expression]) val pivotColumn = if (ctx.pivotColumn.identifiers.size == 1) { - UnresolvedAttribute.quoted(ctx.pivotColumn.errorCapturingIdentifier.getText) + UnresolvedAttribute.quoted(getIdentifierText(ctx.pivotColumn.errorCapturingIdentifier)) } else { CreateStruct( ctx.pivotColumn.identifiers.asScala.map( - identifier => UnresolvedAttribute.quoted(identifier.getText)).toSeq) + identifier => UnresolvedAttribute.quoted(getIdentifierText(identifier))).toSeq) } val pivotValues = ctx.pivotValues.asScala.map(visitPivotValue) Pivot(None, pivotColumn, pivotValues.toSeq, aggregates, query) @@ -1974,7 +1974,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitPivotValue(ctx: PivotValueContext): Expression = withOrigin(ctx) { val e = expression(ctx.expression) if (ctx.errorCapturingIdentifier != null) { - Alias(e, ctx.errorCapturingIdentifier.getText)() + Alias(e, getIdentifierText(ctx.errorCapturingIdentifier))() } else { e } @@ -2039,7 +2039,7 @@ class AstBuilder extends DataTypeAstBuilder // alias unpivot result if (ctx.errorCapturingIdentifier() != null) { - val alias = ctx.errorCapturingIdentifier().getText + val alias = getIdentifierText(ctx.errorCapturingIdentifier()) SubqueryAlias(alias, filtered) } else { filtered @@ -2541,7 +2541,7 @@ class AstBuilder extends DataTypeAstBuilder */ private def mayApplyAliasPlan(tableAlias: TableAliasContext, plan: LogicalPlan): LogicalPlan = { if (tableAlias.strictIdentifier != null) { - val alias = tableAlias.strictIdentifier.getText + val alias = getIdentifierText(tableAlias.strictIdentifier) if (tableAlias.identifierList != null) { val columnNames = visitIdentifierList(tableAlias.identifierList) SubqueryAlias(alias, UnresolvedSubqueryColumnAliases(columnNames, plan)) @@ -3229,7 +3229,7 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { val arguments = ctx.identifier().asScala.map { name => - UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) + UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(getIdentifierText(name)).nameParts) } val function = expression(ctx.expression).transformUp { case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) @@ -4261,7 +4261,7 @@ class AstBuilder extends DataTypeAstBuilder if (!SQLConf.get.objectLevelCollationsEnabled) { throw QueryCompilationErrors.objectLevelCollationsNotEnabledError() } - val collationName = ctx.identifier.getText + val collationName = getIdentifierText(ctx.identifier) CollationFactory.fetchCollation(collationName).collationName } @@ -4500,7 +4500,7 @@ class AstBuilder extends DataTypeAstBuilder def getFieldReference( ctx: ApplyTransformContext, arg: V2Expression): FieldReference = { - lazy val name: String = ctx.identifier.getText + lazy val name: String = getIdentifierText(ctx.identifier) arg match { case ref: FieldReference => ref @@ -4512,7 +4512,7 @@ class AstBuilder extends DataTypeAstBuilder def getSingleFieldReference( ctx: ApplyTransformContext, arguments: Seq[V2Expression]): FieldReference = { - lazy val name: String = ctx.identifier.getText + lazy val name: String = getIdentifierText(ctx.identifier) if (arguments.size > 1) { throw QueryParsingErrors.wrongNumberArgumentsForTransformError(name, arguments.size, ctx) } else if (arguments.isEmpty) { @@ -4797,7 +4797,7 @@ class AstBuilder extends DataTypeAstBuilder string(visitStringLit(c.outFmt))))) // Expected format: SEQUENCEFILE | TEXTFILE | RCFILE | ORC | PARQUET | AVRO case (c: GenericFileFormatContext, null) => - SerdeInfo(storedAs = Some(c.identifier.getText)) + SerdeInfo(storedAs = Some(c.simpleIdentifier.getText)) case (null, storageHandler) => invalidStatement("STORED BY", ctx) case _ => @@ -4887,7 +4887,7 @@ class AstBuilder extends DataTypeAstBuilder (rowFormatCtx, createFileFormatCtx.fileFormat) match { case (_, ffTable: TableFileFormatContext) => // OK case (rfSerde: RowFormatSerdeContext, ffGeneric: GenericFileFormatContext) => - ffGeneric.identifier.getText.toLowerCase(Locale.ROOT) match { + ffGeneric.simpleIdentifier.getText.toLowerCase(Locale.ROOT) match { case ("sequencefile" | "textfile" | "rcfile") => // OK case fmt => operationNotAllowed( @@ -4895,7 +4895,7 @@ class AstBuilder extends DataTypeAstBuilder parentCtx) } case (rfDelimited: RowFormatDelimitedContext, ffGeneric: GenericFileFormatContext) => - ffGeneric.identifier.getText.toLowerCase(Locale.ROOT) match { + ffGeneric.simpleIdentifier.getText.toLowerCase(Locale.ROOT) match { case "textfile" => // OK case fmt => operationNotAllowed( s"ROW FORMAT DELIMITED is only compatible with 'textfile', not '$fmt'", parentCtx) @@ -5837,9 +5837,9 @@ class AstBuilder extends DataTypeAstBuilder log"${MDC(PARTITION_SPECIFICATION, ctx.partitionSpec.getText)}") } } - if (ctx.identifier != null && - ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { - throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.identifier()) + if (ctx.simpleIdentifier != null && + ctx.simpleIdentifier.getText.toLowerCase(Locale.ROOT) != "noscan") { + throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.simpleIdentifier) } if (ctx.ALL() != null) { @@ -5860,7 +5860,7 @@ class AstBuilder extends DataTypeAstBuilder "ANALYZE TABLE", allowTempView = false), partitionSpec, - noScan = ctx.identifier != null) + noScan = ctx.simpleIdentifier != null) } else { checkPartitionSpec() AnalyzeColumn( @@ -5878,16 +5878,16 @@ class AstBuilder extends DataTypeAstBuilder * }}} */ override def visitAnalyzeTables(ctx: AnalyzeTablesContext): LogicalPlan = withOrigin(ctx) { - if (ctx.identifier != null && - ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { - throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.identifier()) + if (ctx.simpleIdentifier != null && + ctx.simpleIdentifier.getText.toLowerCase(Locale.ROOT) != "noscan") { + throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.simpleIdentifier()) } val ns = if (ctx.identifierReference() != null) { withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) } else { CurrentNamespace } - AnalyzeTables(ns, noScan = ctx.identifier != null) + AnalyzeTables(ns, noScan = ctx.simpleIdentifier != null) } /** @@ -6433,7 +6433,7 @@ class AstBuilder extends DataTypeAstBuilder * }}} */ override def visitDropIndex(ctx: DropIndexContext): LogicalPlan = withOrigin(ctx) { - val indexName = ctx.identifier.getText + val indexName = getIdentifierText(ctx.identifier) DropIndex( createUnresolvedTable(ctx.identifierReference, "DROP INDEX"), indexName, @@ -6655,7 +6655,7 @@ class AstBuilder extends DataTypeAstBuilder target = None, excepts = ids.map(s => Seq(s)), replacements = None)) Project(projectList, left) }.getOrElse(Option(ctx.AS).map { _ => - SubqueryAlias(ctx.errorCapturingIdentifier().getText, left) + SubqueryAlias(getIdentifierText(ctx.errorCapturingIdentifier()), left) }.getOrElse(Option(ctx.whereClause).map { c => if (ctx.windowClause() != null) { throw QueryParsingErrors.windowClauseInPipeOperatorWhereClauseNotAllowedError(ctx) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index f8f6e31be1bcc..d1e021d564c3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -187,7 +187,7 @@ class SparkSqlAstBuilder extends AstBuilder { (ident, _) => builder(ident)) } else if (ctx.errorCapturingIdentifier() != null) { // resolve immediately - builder.apply(Seq(ctx.errorCapturingIdentifier().getText)) + builder.apply(Seq(getIdentifierText(ctx.errorCapturingIdentifier()))) } else if (ctx.stringLit() != null) { // resolve immediately builder.apply(Seq(string(visitStringLit(ctx.stringLit())))) @@ -567,7 +567,7 @@ class SparkSqlAstBuilder extends AstBuilder { * - '/path/to/fileOrJar' */ override def visitManageResource(ctx: ManageResourceContext): LogicalPlan = withOrigin(ctx) { - val rawArg = remainder(ctx.identifier).trim + val rawArg = remainder(ctx.simpleIdentifier).trim val maybePaths = strLiteralDef.findAllIn(rawArg).toSeq.map { case p if p.startsWith("\"") || p.startsWith("'") => unescapeSQLString(p) case p => p @@ -575,14 +575,14 @@ class SparkSqlAstBuilder extends AstBuilder { ctx.op.getType match { case SqlBaseParser.ADD => - ctx.identifier.getText.toLowerCase(Locale.ROOT) match { + ctx.simpleIdentifier.getText.toLowerCase(Locale.ROOT) match { case "files" | "file" => AddFilesCommand(maybePaths) case "jars" | "jar" => AddJarsCommand(maybePaths) case "archives" | "archive" => AddArchivesCommand(maybePaths) case other => operationNotAllowed(s"ADD with resource type '$other'", ctx) } case SqlBaseParser.LIST => - ctx.identifier.getText.toLowerCase(Locale.ROOT) match { + ctx.simpleIdentifier.getText.toLowerCase(Locale.ROOT) match { case "files" | "file" => if (maybePaths.length > 0) { ListFilesCommand(maybePaths) @@ -735,7 +735,7 @@ class SparkSqlAstBuilder extends AstBuilder { */ override def visitCreateFunction(ctx: CreateFunctionContext): LogicalPlan = withOrigin(ctx) { val resources = ctx.resource.asScala.map { resource => - val resourceType = resource.identifier.getText.toLowerCase(Locale.ROOT) + val resourceType = resource.simpleIdentifier.getText.toLowerCase(Locale.ROOT) resourceType match { case "jar" | "file" | "archive" => FunctionResource(FunctionResourceType.fromString(resourceType), @@ -1308,7 +1308,7 @@ class SparkSqlAstBuilder extends AstBuilder { } else { DescribeColumn( relation, - UnresolvedAttribute(ctx.describeColName.nameParts.asScala.map(_.getText).toSeq), + UnresolvedAttribute(ctx.describeColName.nameParts.asScala.map(getIdentifierText).toSeq), isExtended) } } else { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out index e53a8153e8292..94fff8f586972 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -2436,6 +2436,47 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, false + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE describe_col_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.describe_col_test + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index abc6cc625b6aa..e6a406072c48b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -2021,22 +2021,11 @@ Project [map(mykey, 42)[mykey] AS result#x] EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' USING 't' AS alias -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`t`.`c1`", - "proposal" : "`IDENTIFIER('t')`.`c1`, `IDENTIFIER('t')`.`c2`, `IDENTIFIER('t')`.`c4`" - }, - "queryContext" : [ { - "objectType" : "EXECUTE IMMEDIATE", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 31, - "fragment" : "IDENTIFIER(:alias '.c1')" - } ] -} +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x] + +- SubqueryAlias t + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x,c4#x] csv -- !query @@ -2182,6 +2171,31 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, false + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query analysis +DescribeColumnCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, [spark_catalog, identifier_clause_test_schema, describe_col_test, c1], false, [info_name#x, info_value#x] + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query analysis +DescribeColumnCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, [spark_catalog, identifier_clause_test_schema, describe_col_test, c2], false, [info_name#x, info_value#x] + + +-- !query +DROP TABLE describe_col_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.describe_col_test + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index bb6c7107062d8..d9bafe7cc607e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -367,6 +367,12 @@ SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL; DROP TABLE unpivot_test; +-- DESCRIBE column with IDENTIFIER() +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV; +DESCRIBE describe_col_test IDENTIFIER('c1'); +DESCRIBE describe_col_test IDENTIFIER('c2'); +DROP TABLE describe_col_test; + -- All the following tests fail because they are not about "true" identifiers -- This should fail - named parameters don't support IDENTIFIER() diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out index 272d9bce81653..6a99be0570100 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -2740,6 +2740,54 @@ struct<> +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE describe_col_test +-- !query schema +struct<> +-- !query output + + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index b398c07b14e11..0c0473791201f 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -2227,24 +2227,11 @@ struct EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' USING 't' AS alias -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`t`.`c1`", - "proposal" : "`IDENTIFIER('t')`.`c1`, `IDENTIFIER('t')`.`c2`, `IDENTIFIER('t')`.`c4`" - }, - "queryContext" : [ { - "objectType" : "EXECUTE IMMEDIATE", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 31, - "fragment" : "IDENTIFIER(:alias '.c1')" - } ] -} +1 +2 +3 -- !query @@ -2396,6 +2383,42 @@ struct<> +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query schema +struct +-- !query output +col_name c1 +data_type int +comment NULL + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query schema +struct +-- !query output +col_name c2 +data_type string +comment NULL + + +-- !query +DROP TABLE describe_col_test +-- !query schema +struct<> +-- !query output + + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/IdentifierClauseParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/IdentifierClauseParserSuite.scala new file mode 100644 index 0000000000000..c2efc43ae576b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/IdentifierClauseParserSuite.scala @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, ExpressionWithUnresolvedIdentifier, UnresolvedAttribute, UnresolvedExtractValue, UnresolvedFunction, UnresolvedInlineTable, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, LambdaFunction, Literal, UnresolvedNamedLambdaVariable} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, OneRowRelation, Pivot, Project, SubqueryAlias, Unpivot} +import org.apache.spark.sql.catalyst.util.EvaluateUnresolvedInlineTable +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +class IdentifierClauseParserSuite extends AnalysisTest { + + import org.apache.spark.sql.catalyst.dsl.expressions._ + import org.apache.spark.sql.catalyst.dsl.plans._ + + private def intercept(sqlCommand: String): ParseException = { + intercept[ParseException](parsePlan(sqlCommand)) + } + + test("UNPIVOT column alias with IDENTIFIER()") { + comparePlans( + parsePlan( + "SELECT * FROM unpivot_test UNPIVOT (val FOR col IN " + + "(a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b')))"), + Unpivot( + None, + Some(Seq(Seq($"a"), Seq($"b"))), + Some(Seq(Some("col_a"), Some("col_b"))), + "col", + Seq("val"), + table("unpivot_test")) + .where(coalesce($"val").isNotNull) + .select(star()) + ) + } + + test("UNPIVOT multi-value column alias with IDENTIFIER()") { + comparePlans( + parsePlan( + "SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN " + + "((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc')))"), + Unpivot( + None, + Some(Seq(Seq($"a", $"b"), Seq($"b", $"c"))), + Some(Seq(Some("cols_ab"), Some("cols_bc"))), + "col", + Seq("v1", "v2"), + table("unpivot_test")) + .where(coalesce($"v1", $"v2").isNotNull) + .select(star()) + ) + } + + test("PIVOT column with IDENTIFIER()") { + comparePlans( + parsePlan( + "SELECT * FROM pivot_test PIVOT (SUM(revenue) FOR IDENTIFIER('quarter') IN ('Q1', 'Q2'))"), + Pivot( + None, + UnresolvedAttribute.quoted("quarter"), + Seq(Literal("Q1"), Literal("Q2")), + Seq(UnresolvedFunction("SUM", Seq($"revenue"), isDistinct = false)), + table("pivot_test")) + .select(star()) + ) + } + + test("PIVOT value alias with IDENTIFIER()") { + comparePlans( + parsePlan( + "SELECT * FROM pivot_test PIVOT (SUM(revenue) AS IDENTIFIER('total') FOR quarter IN " + + "('Q1' AS IDENTIFIER('first_quarter'), 'Q2' AS IDENTIFIER('second_quarter')))"), + Pivot( + None, + $"quarter", + Seq( + Alias(Literal("Q1"), "first_quarter")(), + Alias(Literal("Q2"), "second_quarter")() + ), + Seq(Alias(UnresolvedFunction("SUM", Seq($"revenue"), isDistinct = false), "total")()), + table("pivot_test")) + .select(star()) + ) + } + + test("Lambda variable name with IDENTIFIER()") { + val lambdaVar = UnresolvedNamedLambdaVariable(Seq("x")) + comparePlans( + parsePlan("SELECT transform(array(1, 2, 3), IDENTIFIER('x') -> x + 1)"), + OneRowRelation() + .select( + UnresolvedFunction( + "transform", + Seq( + UnresolvedFunction( + "array", + Seq(Literal(1), Literal(2), Literal(3)), + isDistinct = false), + LambdaFunction( + lambdaVar + Literal(1), + Seq(lambdaVar) + ) + ), + isDistinct = false + ) + ) + ) + } + + test("Struct field names with IDENTIFIER() in CAST") { + val structType = StructType(Seq( + StructField("field1", IntegerType), + StructField("field2", StringType) + )) + comparePlans( + parsePlan( + "SELECT CAST(named_struct('field1', 1, 'field2', 'a') AS " + + "STRUCT)"), + OneRowRelation() + .select( + Cast( + UnresolvedFunction( + "named_struct", + Seq(Literal("field1"), Literal(1), Literal("field2"), Literal("a")), + isDistinct = false), + structType + ) + ) + ) + } + + test("Struct field access with IDENTIFIER()") { + val plan = parsePlan("SELECT IDENTIFIER('data').IDENTIFIER('field1') FROM struct_field_test") + val resolvedPlan = plan.transformAllExpressions { + case e: ExpressionWithUnresolvedIdentifier => + e.exprBuilder(Seq(e.identifierExpr.eval().toString), e.otherExprs) + } + + comparePlans( + resolvedPlan, + table("struct_field_test").select(UnresolvedExtractValue($"data", Literal("field1"))) + ) + } + + test("Struct field access with multiple IDENTIFIER() parts") { + val plan = parsePlan("SELECT IDENTIFIER('a').IDENTIFIER('b').IDENTIFIER('c') FROM t") + val resolvedPlan = plan.transformAllExpressions { + case e: ExpressionWithUnresolvedIdentifier => + e.exprBuilder(Seq(e.identifierExpr.eval().toString), e.otherExprs) + } + + comparePlans( + resolvedPlan, + table("t").select( + UnresolvedExtractValue( + UnresolvedExtractValue($"a", Literal("b")), + Literal("c") + ) + ) + ) + } + + test("Partition spec with IDENTIFIER() for partition column name") { + val plan = parsePlan( + "INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value1') VALUES (1)") + .asInstanceOf[InsertIntoStatement] + val values = EvaluateUnresolvedInlineTable.evaluate( + UnresolvedInlineTable(Seq("col1"), Seq(Seq(Literal(1))))) + + comparePlans( + plan, + InsertIntoStatement( + plan.table, + Map("c2" -> Some("value1")), + Nil, + values, + overwrite = false, + ifPartitionNotExists = false + ) + ) + } + + test("Pipe operator alias with IDENTIFIER()") { + val values = EvaluateUnresolvedInlineTable.evaluate( + UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Literal(1), Literal(2))))) + comparePlans( + parsePlan( + "SELECT * FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('pipe_alias') |> SELECT c1, c2"), + Project( + Seq($"c1", $"c2"), + SubqueryAlias( + "pipe_alias", + Project( + Seq(UnresolvedStar(None)), + SubqueryAlias("T", values) + ) + ) + ) + ) + } + + test("Pipe operator alias with IDENTIFIER() - second variant") { + val values = EvaluateUnresolvedInlineTable.evaluate( + UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Literal(1), Literal(2))))) + comparePlans( + parsePlan( + "SELECT c1, c2 FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('my_result') |> SELECT *"), + Project( + Seq(UnresolvedStar(None)), + SubqueryAlias( + "my_result", + Project( + Seq($"c1", $"c2"), + SubqueryAlias("T", values) + ) + ) + ) + ) + } + + test("Resource type ADD is a keyword - should fail") { + checkError( + exception = intercept("ADD IDENTIFIER('file') '/tmp/test.txt'"), + condition = "INVALID_SQL_SYNTAX.UNSUPPORTED_SQL_STATEMENT", + parameters = Map("sqlText" -> "ADD IDENTIFIER('file') '/tmp/test.txt'"), + context = ExpectedContext( + fragment = "ADD IDENTIFIER('file') '/tmp/test.txt'", + start = 0, + stop = 37 + ) + ) + } + + test("Resource type LIST is a keyword - should fail") { + checkError( + exception = intercept("LIST IDENTIFIER('files')"), + condition = "INVALID_SQL_SYNTAX.UNSUPPORTED_SQL_STATEMENT", + parameters = Map("sqlText" -> "LIST IDENTIFIER('files')"), + context = ExpectedContext( + fragment = "LIST IDENTIFIER('files')", + start = 0, + stop = 23 + ) + ) + } + + test("CREATE FUNCTION USING resource type is a keyword - should fail") { + checkError( + exception = intercept( + "CREATE FUNCTION keyword_test_func AS 'com.example.Test' " + + "USING IDENTIFIER('jar') '/path/to.jar'"), + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'('", "hint" -> "") + ) + } + + test("ANALYZE TABLE NOSCAN is a keyword - should fail") { + checkError( + exception = intercept( + "ANALYZE TABLE analyze_keyword_test COMPUTE STATISTICS IDENTIFIER('noscan')"), + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'('", "hint" -> "") + ) + } +} From 41127db2e17d305bd829ff7b90c3cfe00c4fd055 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Herman=20van=20H=C3=B6vell?= Date: Fri, 19 Dec 2025 23:05:23 +0800 Subject: [PATCH 310/400] [SPARK-54696][CONNECT] Clean-up Arrow Buffers - follow-up MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? There were a couple of ommissions in https://github.com/apache/spark/commit/c36b7e58d0422a13228252657e4cff26a762a228 this PR addresses them. The following changes were made: - Testing that arrow buffers are actually cleaned up when IPC stream iterators are exhausted. - Throw a proper error when there is a schema mismatch in between different IPC streams. - Tidy up some duplicate code in the SparkConnectPlanner. ### Why are the changes needed? The previous PR was merged in a hurry. These things were missed. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? It adds tests for IPC stream iterators. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53480 from hvanhovell/SPARK-54696-follow-up-2. Authored-by: Herman van Hövell Signed-off-by: Wenchen Fan (cherry picked from commit 09a2cadc1fb4c162565bb70610867d6f1aa10dee) Signed-off-by: Wenchen Fan --- .../sql/util/ConcatenatingArrowStreamReader.scala | 4 +++- .../sql/connect/planner/InvalidInputErrors.scala | 3 --- .../spark/sql/execution/arrow/ArrowConverters.scala | 4 +++- .../sql/execution/arrow/ArrowConvertersSuite.scala | 12 ++++++++++++ 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala index 5de53a568a7d3..2e5706fe4dcca 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ConcatenatingArrowStreamReader.scala @@ -25,6 +25,8 @@ import org.apache.arrow.vector.ipc.{ArrowReader, ReadChannel} import org.apache.arrow.vector.ipc.message.{ArrowDictionaryBatch, ArrowMessage, ArrowRecordBatch, MessageChannelReader, MessageResult, MessageSerializer} import org.apache.arrow.vector.types.pojo.Schema +import org.apache.spark.SparkException + /** * An [[ArrowReader]] that concatenates multiple [[MessageIterator]]s into a single stream. Each * iterator represents a single IPC stream. The concatenated streams all must have the same @@ -62,7 +64,7 @@ private[sql] class ConcatenatingArrowStreamReader( totalBytesRead += current.bytesRead current = input.next() if (current.schema != getVectorSchemaRoot.getSchema) { - throw new IllegalStateException() + throw SparkException.internalError("IPC Streams have different schemas.") } } if (current.hasNext) { diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala index fcef696c88afc..eb4df9673e594 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala @@ -96,9 +96,6 @@ object InvalidInputErrors { def chunkedCachedLocalRelationWithoutData(): InvalidPlanInput = InvalidPlanInput("ChunkedCachedLocalRelation should contain data.") - def chunkedCachedLocalRelationChunksWithDifferentSchema(): InvalidPlanInput = - InvalidPlanInput("ChunkedCachedLocalRelation data chunks have different schema.") - def schemaRequiredForLocalRelation(): InvalidPlanInput = InvalidPlanInput("Schema for LocalRelation is required when the input data is not provided.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index e227a58528721..3d435ac016ed1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -314,7 +314,7 @@ private[sql] object ArrowConverters extends Logging { new ConcatenatingArrowStreamReader(allocator, messages, destructive = true) } - val schema: StructType = try { + lazy val schema: StructType = try { ArrowUtils.fromArrowSchema(reader.getVectorSchemaRoot.getSchema) } catch { case NonFatal(e) => @@ -338,6 +338,8 @@ private[sql] object ArrowConverters extends Logging { // Public accessors for metrics def batchesLoaded: Int = _batchesLoaded def totalRowsProcessed: Long = _totalRowsProcessed + def allocatedMemory: Long = allocator.getAllocatedMemory + def peakMemoryAllocation: Long = allocator.getPeakMemoryAllocation override def hasNext: Boolean = { while (!rowIterator.hasNext) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala index f58a5b7ebd6a4..95cd97c2c742d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala @@ -1795,7 +1795,10 @@ class ArrowConvertersSuite extends SharedSparkSession { val (outputRowIter, outputSchema) = ArrowConverters. fromIPCStreamWithIterator(out.toByteArray, null) assert(outputSchema == schema) + assert(outputRowIter.peakMemoryAllocation == 0) val outputRows = outputRowIter.map(proj(_).copy()).toList + assert(outputRowIter.peakMemoryAllocation > 0) + assert(outputRowIter.allocatedMemory == 0) assert(outputRows.length == inputRows.length) outputRows.zipWithIndex.foreach { case (row, i) => assert(row.getInt(0) == i) @@ -1820,6 +1823,7 @@ class ArrowConvertersSuite extends SharedSparkSession { val (iterator, outputSchema) = ArrowConverters.fromIPCStreamWithIterator(out.toByteArray, ctx) assert(outputSchema == schema) + assert(iterator.peakMemoryAllocation == 0) // Initially no batches loaded assert(iterator.batchesLoaded == 0) @@ -1837,6 +1841,8 @@ class ArrowConvertersSuite extends SharedSparkSession { // Consume all rows val proj = UnsafeProjection.create(schema) val outputRows = iterator.map(proj(_).copy()).toList + assert(iterator.peakMemoryAllocation > 0) + assert(iterator.allocatedMemory == 0) assert(outputRows.length == inputRows.length) outputRows.zipWithIndex.foreach { case (row, i) => assert(row.getInt(0) == i) @@ -1889,9 +1895,12 @@ class ArrowConvertersSuite extends SharedSparkSession { val (outputRowIter, outputSchema) = ArrowConverters. fromIPCStreamWithIterator(out.toByteArray, ctx) + assert(outputRowIter.peakMemoryAllocation == 0) val proj = UnsafeProjection.create(schema) assert(outputSchema == schema) val outputRows = outputRowIter.map(proj(_).copy()).toList + assert(outputRowIter.peakMemoryAllocation > 0) + assert(outputRowIter.allocatedMemory == 0) assert(outputRows.length == inputRows.length) outputRows.zipWithIndex.foreach { case (row, i) => @@ -1926,6 +1935,7 @@ class ArrowConvertersSuite extends SharedSparkSession { val (iterator, outputSchema) = ArrowConverters.fromIPCStreamWithIterator(out.toByteArray, ctx) assert(outputSchema == schema) + assert(iterator.peakMemoryAllocation == 0) // Initially no batches loaded assert(iterator.batchesLoaded == 0) @@ -1945,6 +1955,8 @@ class ArrowConvertersSuite extends SharedSparkSession { val remainingRows = iterator.toList val totalConsumed = firstBatch.length + remainingRows.length assert(totalConsumed == inputRows.length) + assert(iterator.peakMemoryAllocation > 0) + assert(iterator.allocatedMemory == 0) // Final metrics should show all batches loaded val expectedBatches = Math.ceil(inputRows.length.toDouble / batchSize).toInt From 1a6512205ea0dea26704b3aabce9a8d51a68e6a7 Mon Sep 17 00:00:00 2001 From: Tengfei Huang Date: Sat, 20 Dec 2025 00:03:08 +0800 Subject: [PATCH 311/400] [SPARK-54556][CORE] Rollback succeeding shuffle map stages when shuffle checksum mismatch detected ### What changes were proposed in this pull request? Rollback shuffle map stages when shuffle checksum mismatch detected: - cancel and resubmit the stage if it's running; - clean up the shuffle status to ensure it'll be resubmitted; - mark rollback attemptId and ignore the results from these elder attempts which may consume inconsistent data; ### Why are the changes needed? To ensure all the succeeding stages will be re-submitted and fully-retry when there is shuffle checksum mismatch detected. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT added. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53274 from ivoson/SPARK-54556. Authored-by: Tengfei Huang Signed-off-by: Wenchen Fan (cherry picked from commit 0da9e0505b59d145c6429a3113409dc33f9a1efc) Signed-off-by: Wenchen Fan --- .../apache/spark/scheduler/DAGScheduler.scala | 221 ++++++++++++++---- .../org/apache/spark/scheduler/Stage.scala | 16 ++ .../spark/scheduler/DAGSchedulerSuite.scala | 130 ++++++++++- 3 files changed, 313 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7c8bea31334b1..69e766ebcef25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1560,42 +1560,27 @@ private[spark] class DAGScheduler( // `findMissingPartitions()` returns all partitions every time. stage match { case sms: ShuffleMapStage if !sms.isAvailable => - val needFullStageRetry = if (sms.shuffleDep.checksumMismatchFullRetryEnabled) { - // When the parents of this stage are indeterminate (e.g., some parents are not - // checkpointed and checksum mismatches are detected), the output data of the parents - // may have changed due to task retries. For correctness reason, we need to - // retry all tasks of the current stage. The legacy way of using current stage's - // deterministic level to trigger full stage retry is not accurate. - stage.isParentIndeterminate - } else { - if (stage.isIndeterminate) { - // already executed at least once - if (sms.getNextAttemptId > 0) { - // While we previously validated possible rollbacks during the handling of a FetchFailure, - // where we were fetching from an indeterminate source map stages, this later check - // covers additional cases like recalculating an indeterminate stage after an executor - // loss. Moreover, because this check occurs later in the process, if a result stage task - // has successfully completed, we can detect this and abort the job, as rolling back a - // result stage is not possible. - val stagesToRollback = collectSucceedingStages(sms) - abortStageWithInvalidRollBack(stagesToRollback) - // stages which cannot be rolled back were aborted which leads to removing the - // the dependant job(s) from the active jobs set - val numActiveJobsWithStageAfterRollback = - activeJobs.count(job => stagesToRollback.contains(job.finalStage)) - if (numActiveJobsWithStageAfterRollback == 0) { - logInfo(log"All jobs depending on the indeterminate stage " + - log"(${MDC(STAGE_ID, stage.id)}) were aborted so this stage is not needed anymore.") - return - } + if (!sms.shuffleDep.checksumMismatchFullRetryEnabled && stage.isIndeterminate) { + // already executed at least once + if (sms.getNextAttemptId > 0) { + // While we previously validated possible rollbacks during the handling of a FetchFailure, + // where we were fetching from an indeterminate source map stages, this later check + // covers additional cases like recalculating an indeterminate stage after an executor + // loss. Moreover, because this check occurs later in the process, if a result stage task + // has successfully completed, we can detect this and abort the job, as rolling back a + // result stage is not possible. + val stagesToRollback = collectSucceedingStages(sms) + filterAndAbortUnrollbackableStages(stagesToRollback) + // stages which cannot be rolled back were aborted which leads to removing the + // the dependant job(s) from the active jobs set + val numActiveJobsWithStageAfterRollback = + activeJobs.count(job => stagesToRollback.contains(job.finalStage)) + if (numActiveJobsWithStageAfterRollback == 0) { + logInfo(log"All jobs depending on the indeterminate stage " + + log"(${MDC(STAGE_ID, stage.id)}) were aborted so this stage is not needed anymore.") + return } - true - } else { - false } - } - - if (needFullStageRetry) { mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) sms.shuffleDep.newShuffleMergeState() } @@ -1913,16 +1898,127 @@ private[spark] class DAGScheduler( /** * If a map stage is non-deterministic, the map tasks of the stage may return different result - * when re-try. To make sure data correctness, we need to re-try all the tasks of its succeeding - * stages, as the input data may be changed after the map tasks are re-tried. For stages where - * rollback and retry all tasks are not possible, we will need to abort the stages. + * when re-try. To make sure data correctness, we need to clean up shuffles to make sure succeeding + * stages will be resubmitted and re-try all the tasks, as the input data may be changed after + * the map tasks are re-tried. For stages where rollback and retry all tasks are not possible, + * we will need to abort the stages. + */ + private[scheduler] def rollbackSucceedingStages(mapStage: ShuffleMapStage): Unit = { + val stagesToRollback = collectSucceedingStages(mapStage).filterNot(_ == mapStage) + val stagesCanRollback = filterAndAbortUnrollbackableStages(stagesToRollback) + // stages which cannot be rolled back were aborted which leads to removing the + // the dependant job(s) from the active jobs set, there could be no active jobs + // left depending on the indeterminate stage and hence no need to roll back any stages. + val numActiveJobsWithStageAfterRollback = + activeJobs.count(job => stagesToRollback.contains(job.finalStage)) + if (numActiveJobsWithStageAfterRollback == 0) { + logInfo(log"All jobs depending on the indeterminate stage " + + log"(${MDC(STAGE_ID, mapStage.id)}) were aborted.") + } else { + // Mark rollback attempt to identify elder attempts which could consume inconsistent data, + // the results from these attempts should be ignored. + // Rollback the running stages first to avoid triggering more fetch failures. + stagesToRollback.toSeq.sortBy(!runningStages.contains(_)).foreach { + case sms: ShuffleMapStage => + rollbackShuffleMapStage(sms, "rolling back due to indeterminate " + + s"output of shuffle map stage $mapStage") + sms.markAsRollingBack() + + case rs: ResultStage => + rs.markAsRollingBack() + } + + logInfo(log"The shuffle map stage ${MDC(STAGE, mapStage)} with indeterminate output " + + log"was retried, we will roll back and rerun its succeeding " + + log"stages: ${MDC(STAGES, stagesCanRollback)}") + } + } + + /** + * Roll back the given shuffle map stage: + * 1. If the stage is running, cancel the stage and kill all running tasks. Clean up the shuffle + * output resubmit it if it's not exceeded max retries. + * 2. If the stage is not running but having output generated, clean up the shuffle output to + * ensure the stage will be re-executed with fully retry. + * + * @param sms the shuffle map stage to roll back + * @param reason the reason for rolling back + */ + private def rollbackShuffleMapStage(sms: ShuffleMapStage, reason: String): Unit = { + logInfo(log"Rolling back ${MDC(STAGE, sms)} due to indeterminate rollback") + val clearShuffle = if (runningStages.contains(sms)) { + logInfo(log"Stage ${MDC(STAGE, sms)} is running, marking it as failed and " + + log"resubmit if allowed") + cancelStageAndTryResubmit(sms, reason) + } else { + true + } + + // Clean up shuffle outputs in case the stage is not aborted to ensure the stage + // will be re-executed. + if (clearShuffle) { + logInfo(log"Cleaning up shuffle for stage ${MDC(STAGE, sms)} to ensure re-execution") + mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) + sms.shuffleDep.newShuffleMergeState() + } + } + + /** + * Cancel the give running shuffle map stage, killing all running tasks, resubmit if it doesn't + * exceed max retries. + * + * @param stage the stage to cancel and resubmit + * @param reason the reason for the operation + * @return true if the stage is successfully cancelled and resubmitted, otherwise false */ - private[scheduler] def abortUnrollbackableStages(mapStage: ShuffleMapStage): Unit = { - val stagesToRollback = collectSucceedingStages(mapStage) - val rollingBackStages = abortStageWithInvalidRollBack(stagesToRollback) - logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output " + - log"was failed, we will roll back and rerun below stages which include itself and all its " + - log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") + private def cancelStageAndTryResubmit(stage: ShuffleMapStage, reason: String): Boolean = { + assert(runningStages.contains(stage), "stage must be running to be cancelled and resubmitted") + try { + // killAllTaskAttempts will fail if a SchedulerBackend does not implement killTask. + val job = jobIdToActiveJob.get(stage.firstJobId) + val shouldInterrupt = job.exists(j => shouldInterruptTaskThread(j)) + taskScheduler.killAllTaskAttempts(stage.id, shouldInterrupt, reason) + } catch { + case e: UnsupportedOperationException => + logWarning(log"Could not kill all tasks for stage ${MDC(STAGE_ID, stage.id)}", e) + abortStage(stage, "Rollback failed due to: Not able to kill running tasks for stage " + + s"$stage (${stage.name})", Some(e)) + return false + } + + stage.failedAttemptIds.add(stage.latestInfo.attemptNumber()) + val shouldAbortStage = stage.failedAttemptIds.size >= maxConsecutiveStageAttempts || + disallowStageRetryForTest + markStageAsFinished(stage, Some(reason), willRetry = !shouldAbortStage) + + if (shouldAbortStage) { + val abortMessage = if (disallowStageRetryForTest) { + "Stage will not retry stage due to testing config. Most recent failure " + + s"reason: $reason" + } else { + s"$stage (${stage.name}) has failed the maximum allowable number of " + + s"times: $maxConsecutiveStageAttempts. Most recent failure reason: $reason" + } + abortStage(stage, s"rollback failed due to: $abortMessage", None) + } else { + // In case multiple task failures triggered for a single stage attempt, ensure we only + // resubmit the failed stage once. + val noResubmitEnqueued = !failedStages.contains(stage) + failedStages += stage + if (noResubmitEnqueued) { + logInfo(log"Resubmitting ${MDC(FAILED_STAGE, stage)} " + + log"(${MDC(FAILED_STAGE_NAME, stage.name)}) due to rollback.") + messageScheduler.schedule( + new Runnable { + override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) + }, + DAGScheduler.RESUBMIT_TIMEOUT, + TimeUnit.MILLISECONDS + ) + } + } + + !shouldAbortStage } /** @@ -1990,7 +2086,21 @@ private[spark] class DAGScheduler( // tasks complete, they still count and we can mark the corresponding partitions as // finished if the stage is determinate. Here we notify the task scheduler to skip running // tasks for the same partition to save resource. - if (!stage.isIndeterminate && task.stageAttemptId < stage.latestInfo.attemptNumber()) { + def stageWithChecksumMismatchFullRetryEnabled(stage: Stage): Boolean = { + stage match { + case s: ShuffleMapStage => s.shuffleDep.checksumMismatchFullRetryEnabled + case _ => stage.parents.exists(stageWithChecksumMismatchFullRetryEnabled) + } + } + + // Ignore task completion for old attempt of indeterminate stage + val ignoreOldTaskAttempts = if (stageWithChecksumMismatchFullRetryEnabled(stage)) { + stage.maxAttemptIdToIgnore.exists(_ >= task.stageAttemptId) + } else { + stage.isIndeterminate && task.stageAttemptId < stage.latestInfo.attemptNumber() + } + + if (!ignoreOldTaskAttempts && task.stageAttemptId < stage.latestInfo.attemptNumber()) { taskScheduler.notifyPartitionCompletion(stageId, task.partitionId) } @@ -2002,6 +2112,13 @@ private[spark] class DAGScheduler( resultStage.activeJob match { case Some(job) => if (!job.finished(rt.outputId)) { + if (ignoreOldTaskAttempts) { + val reason = "Task with indeterminate results from old attempt succeeded, " + + s"aborting the stage $resultStage to ensure data correctness." + abortStage(resultStage, reason, None) + return + } + job.finished(rt.outputId) = true job.numFinished += 1 // If the whole job has finished, remove it @@ -2045,10 +2162,7 @@ private[spark] class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] - // Ignore task completion for old attempt of indeterminate stage - val ignoreIndeterminate = stage.isIndeterminate && - task.stageAttemptId < stage.latestInfo.attemptNumber() - if (!ignoreIndeterminate) { + if (!ignoreOldTaskAttempts) { shuffleStage.pendingPartitions -= task.partitionId val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId @@ -2077,7 +2191,7 @@ private[spark] class DAGScheduler( shuffleStage.maxChecksumMismatchedId = smt.stageAttemptId if (shuffleStage.shuffleDep.checksumMismatchFullRetryEnabled && shuffleStage.isStageIndeterminate) { - abortUnrollbackableStages(shuffleStage) + rollbackSucceedingStages(shuffleStage) } } } @@ -2206,7 +2320,11 @@ private[spark] class DAGScheduler( // guaranteed to be determinate, so the input data of the reducers will not change // even if the map tasks are re-tried. if (mapStage.isIndeterminate && !mapStage.shuffleDep.checksumMismatchFullRetryEnabled) { - abortUnrollbackableStages(mapStage) + val stagesToRollback = collectSucceedingStages(mapStage) + val stagesCanRollback = filterAndAbortUnrollbackableStages(stagesToRollback) + logInfo(log"The shuffle map stage ${MDC(STAGE, mapStage)} with indeterminate output " + + log"was failed, we will roll back and rerun below stages which include itself and all " + + log"its indeterminate child stages: ${MDC(STAGES, stagesCanRollback)}") } // We expect one executor failure to trigger many FetchFailures in rapid succession, @@ -2396,7 +2514,8 @@ private[spark] class DAGScheduler( * @param stagesToRollback stages to roll back * @return Shuffle map stages which need and can be rolled back */ - private def abortStageWithInvalidRollBack(stagesToRollback: HashSet[Stage]): HashSet[Stage] = { + private def filterAndAbortUnrollbackableStages( + stagesToRollback: HashSet[Stage]): HashSet[Stage] = { def generateErrorMessage(stage: Stage): String = { "A shuffle map stage with indeterminate output was failed and retried. " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 9bf604e9a83cf..d8aaea013ee65 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -84,6 +84,14 @@ private[scheduler] abstract class Stage( */ private[scheduler] var maxChecksumMismatchedId: Int = nextAttemptId + /** + * The max attempt id we should ignore results for this stage, indicating there are ancestor + * stages having been detected with checksum mismatches. This stage is probably also + * indeterminate, so we need to avoid completing the stage and the job with incorrect result + * by ignoring the task output from previous attempts which might consume inconsistent data + */ + private[scheduler] var maxAttemptIdToIgnore: Option[Int] = None + val name: String = callSite.shortForm val details: String = callSite.longForm @@ -108,6 +116,14 @@ private[scheduler] abstract class Stage( failedAttemptIds.clear() } + /** Mark the latest attempt as rollback */ + private[scheduler] def markAsRollingBack(): Unit = { + // Only if the stage has been submitted + if (getNextAttemptId > 0) { + maxAttemptIdToIgnore = Some(latestInfo.attemptNumber()) + } + } + /** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */ def makeNewStageAttempt( numPartitionsToCompute: Int, diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 6ec0ea320eaa0..48f1c49e7af23 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3421,11 +3421,12 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti stageId: Int, shuffleId: Int, numTasks: Int = 2, - checksumVal: Long = 0): Unit = { + checksumVal: Long = 0, + stageAttemptId: Int = 1): Unit = { assert(taskSets(taskSetIndex).stageId == stageId) - assert(taskSets(taskSetIndex).stageAttemptId == 1) + assert(taskSets(taskSetIndex).stageAttemptId == stageAttemptId) assert(taskSets(taskSetIndex).tasks.length == numTasks) - completeShuffleMapStageSuccessfully(stageId, 1, 2, checksumVal = checksumVal) + completeShuffleMapStageSuccessfully(stageId, stageAttemptId, 2, checksumVal = checksumVal) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) } @@ -3835,6 +3836,129 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } } + test("SPARK-54556: ensure rollback all the succeeding stages and ignore stale task results " + + "when shuffle checksum mismatch detected") { + /** + * Construct the following RDD graph: + * + * ShuffleMapRdd1 (Indeterminate) + * / \ + * ShuffleMapRdd2 \ + * / | + * ShuffleMapRdd3 | + * \ | + * FinalRd + * + * While executing the result stage, shuffle fetch failed on shuffle1 and leading to executor + * loss and some map output of shuffle2 lost. + * Both stage 0 and stage 2 will be submitted. + * Checksum mismatch is detected when retrying stage 0. + * Retry task of stage 2 completed and should be ignored. + */ + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil) + val shuffleDep1 = new ShuffleDependency( + shuffleMapRdd1, + new HashPartitioner(2), + checksumMismatchFullRetryEnabled = true) + val shuffleId1 = shuffleDep1.shuffleId + + val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker) + val shuffleDep2 = new ShuffleDependency( + shuffleMapRdd2, + new HashPartitioner(2), + checksumMismatchFullRetryEnabled = true) + val shuffleId2 = shuffleDep2.shuffleId + + val shuffleMapRdd3 = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) + val shuffleDep3 = new ShuffleDependency( + shuffleMapRdd3, + new HashPartitioner(2), + checksumMismatchFullRetryEnabled = true) + val shuffleId3 = shuffleDep3.shuffleId + + val finalRdd = new MyRDD(sc, 2, List(shuffleDep1, shuffleDep3), tracker = mapOutputTracker) + + // Submit the job and complete the shuffle stages + submit(finalRdd, Array(0, 1)) + completeShuffleMapStageSuccessfully( + 0, 0, 2, Seq("hostA", "hostB"), checksumVal = 100) + completeShuffleMapStageSuccessfully( + 1, 0, 2, Seq("hostC", "hostD"), checksumVal = 200) + completeShuffleMapStageSuccessfully( + 2, 0, 2, Seq("hostB", "hostC"), checksumVal = 300) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + assert(mapOutputTracker.findMissingPartitions(shuffleId3) === Some(Seq.empty)) + + // The first task of result stage 3 failed with FetchFailed. + runEvent(makeCompletionEvent( + taskSets(3).tasks(0), + FetchFailed(makeBlockManagerId("hostB"), shuffleId1, 0L, 0, 0, "ignored"), + null)) + assert(mapOutputTracker.findMissingPartitions(shuffleId3).nonEmpty) + + // Check status for all failedStages. + val failedStages = scheduler.failedStages.toSeq + assert(failedStages.map(_.id) === Seq(0, 3)) + scheduler.resubmitFailedStages() + // Check status for runningStages. + assert(scheduler.runningStages.map(_.id) === Set(0, 2)) + + // Complete the re-attempt of shuffle map stage 0(shuffleId1) with a different checksum. + completeShuffleMapStageSuccessfully(0, 1, 2, checksumVal = 101) + completeShuffleMapStageSuccessfully(2, 1, 2, checksumVal = 300) + // The result of stage 2 should be ignored + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId3) === 0) + scheduler.resubmitFailedStages() + assert(scheduler.runningStages.map(_.id) === Set(1)) + + checkAndCompleteRetryStage(6, 1, shuffleId2, 2, checksumVal = 201) + checkAndCompleteRetryStage(7, 2, shuffleId3, 2, checksumVal = 301, stageAttemptId = 2) + completeAndCheckAnswer(taskSets(8), Seq((Success, 11), (Success, 12)), Map(0 -> 11, 1 -> 12)) + } + + test("SPARK-54556: abort stage if result task from old attempt with indeterminate " + + "result succeeded") { + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil) + val shuffleDep1 = new ShuffleDependency( + shuffleMapRdd1, + new HashPartitioner(2), + checksumMismatchFullRetryEnabled = true) + val shuffleId1 = shuffleDep1.shuffleId + + // Submit a job depending on shuffleDep1 + val finalRdd1 = new MyRDD( + sc, 2, List(shuffleDep1), tracker = mapOutputTracker) + submit(finalRdd1, Array(0, 1)) + + // Finish stage 0. + completeShuffleMapStageSuccessfully( + 0, 0, 2, Seq("hostA", "hostB"), checksumVal = 100) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + // The first task of result stage failed with FetchFailed. + runEvent(makeCompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), + null)) + + // Check status for all failedStages. + val failedStages = scheduler.failedStages.toSeq + assert(failedStages.map(_.id) == Seq(0, 1)) + scheduler.resubmitFailedStages() + + // Complete the shuffle map stage with a different checksum + completeShuffleMapStageSuccessfully(0, 1, 2, checksumVal = 101) + + // Complete the second task of 1st attempt of result stage. + runEvent(makeCompletionEvent( + taskSets(1).tasks(1), + Success, + 42)) + assert(failure != null && failure.getMessage.contains( + "Task with indeterminate results from old attempt succeeded")) + } + test("SPARK-27164: RDD.countApprox on empty RDDs schedules jobs which never complete") { val latch = new CountDownLatch(1) val jobListener = new SparkListener { From 94431221bbf2a951be6bdf6769eb14015f287bf2 Mon Sep 17 00:00:00 2001 From: qindongliang <18210507492@126.com> Date: Sat, 20 Dec 2025 00:39:51 +0800 Subject: [PATCH 312/400] =?UTF-8?q?[SPARK-54750][SQL]=20Fix=20ROUND=20retu?= =?UTF-8?q?rning=20NULL=20for=20Decimal=20values=20with=20l=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Fixed a bug in `RoundBase` where `ROUND` function incorrectly returns `NULL` for certain Decimal values. The issue was caused by using the input decimal's runtime precision instead of the target type's precision when calling `toPrecision()`. For example, `ROUND(PERCENTILE_APPROX(2150 / 1000.0, 0.95), 3)` incorrectly returned `NULL` instead of `2.15`. ### Why are the changes needed? The fix changes `decimal.toPrecision(decimal.precision, s, mode)` to `decimal.toPrecision(p, s, mode)` in both `nullSafeEval` and `doGenCode` methods, where `p` is the target DecimalType's precision. ### Does this PR introduce _any_ user-facing change? Yes, this fixes a bug where ROUND could return NULL for valid decimal inputs. ### How was this patch tested? Added regression test in `ApproximatePercentileQuerySuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53529 from qindongliang/SPARK-54750-fix-round-precision. Lead-authored-by: qindongliang <18210507492@126.com> Co-authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit 1e6d743c7ae235b3a23700d9cfdf924cb5e25d61) Signed-off-by: Kent Yao --- .../sql/catalyst/expressions/mathExpressions.scala | 7 +++---- .../spark/sql/ApproximatePercentileQuerySuite.scala | 12 ++++++++++++ 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index ee3e3e0272767..0643e5fba2f32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1572,7 +1572,7 @@ abstract class RoundBase(child: Expression, scale: Expression, val decimal = input1.asInstanceOf[Decimal] if (_scale >= 0) { // Overflow cannot happen, so no need to control nullOnOverflow - decimal.toPrecision(decimal.precision, s, mode) + decimal.toPrecision(p, s, mode) } else { Decimal(decimal.toBigDecimal.setScale(_scale, mode), p, s) } @@ -1644,10 +1644,9 @@ abstract class RoundBase(child: Expression, scale: Expression, case DecimalType.Fixed(p, s) => if (_scale >= 0) { s""" - ${ev.value} = ${ce.value}.toPrecision(${ce.value}.precision(), $s, - Decimal.$modeStr(), true, null); + ${ev.value} = ${ce.value}.toPrecision($p, $s, Decimal.$modeStr(), true, null); ${ev.isNull} = ${ev.value} == null;""" - } else { + } else { s""" ${ev.value} = new Decimal().set(${ce.value}.toBigDecimal() .setScale(${_scale}, Decimal.$modeStr()), $p, $s); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index 3b987529afcb5..a92218e1f1de4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -370,4 +370,16 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession context = ExpectedContext( "", "", 8, 40, "percentile_approx(col, NULL, 100)")) } + + test("SPARK-54750: percentile_approx returns NULL for certain decimal values") { + // Regression test: ROUND(PERCENTILE_APPROX(2150/1000.0, 0.95), 3) should return 2.15 + checkAnswer( + spark.sql("SELECT ROUND(PERCENTILE_APPROX(2150 / 1000.0, 0.95), 3) as p95"), + Row(2.15) + ) + checkAnswer( + spark.sql("SELECT ROUND(PERCENTILE_APPROX(2151 / 1000.0, 0.95), 3) as p95"), + Row(2.151) + ) + } } From 89cdd638b0e282ad1890431e1d60059180efc08f Mon Sep 17 00:00:00 2001 From: yhuang-db Date: Fri, 19 Dec 2025 14:26:11 -0800 Subject: [PATCH 313/400] [SPARK-54761][SQL] Throw unsupportedTableOperation for constraint operation on DSv1/HMS table ### What changes were proposed in this pull request? This PR throws unsupportedTableOperation for constraint operations on DSv1/HMS table. The operations include: - CreateTable/CreateTableAsSelect with constraint - AddConstraint - DropConstraint - AddCheckConstraint ### Why are the changes needed? Constraints (DSv2) are not supported on DSv1/HMS tables. Currently, it does not throw exceptions and causes [confusion](https://github.com/apache/spark/pull/50761#issuecomment-3664237003). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests on dsv1 and hive table. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: claude-4.5-opus Closes #53532 from yhuang-db/SPARK-54761_unsupportedTableOperation-for-constraint-on-dsv1-hms. Authored-by: yhuang-db Signed-off-by: Gengliang Wang (cherry picked from commit 4b90a265c6234570d29d5f4482ddd955107174de) Signed-off-by: Gengliang Wang --- .../analysis/ResolveSessionCatalog.scala | 31 ++++- .../command/v1/TableConstraintSuite.scala | 121 ++++++++++++++++++ .../command/v2/CheckConstraintSuite.scala | 58 +++++---- .../command/v2/DescribeTableSuite.scala | 8 +- .../command/v2/ShowCreateTableSuite.scala | 10 +- .../command/TableConstraintSuite.scala | 26 ++++ 6 files changed, 217 insertions(+), 37 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TableConstraintSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/TableConstraintSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index b1fd0bff1071f..a15ba27962029 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkException import org.apache.spark.internal.LogKeys.CONFIG import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils, ClusterBySpec} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils, ClusterBySpec, HiveTableRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -31,7 +31,7 @@ import org.apache.spark.sql.connector.catalog.{CatalogExtension, CatalogManager, import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1} +import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.internal.connector.V1Function @@ -128,6 +128,25 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case DropColumns(ResolvedV1TableIdentifier(ident), _, _) => throw QueryCompilationErrors.unsupportedTableOperationError(ident, "DROP COLUMN") + // V1 and hive tables do not support constraints + case AddConstraint(ResolvedV1TableIdentifier(ident), _) => + throw QueryCompilationErrors.unsupportedTableOperationError(ident, "ADD CONSTRAINT") + + case DropConstraint(ResolvedV1TableIdentifier(ident), _, _, _) => + throw QueryCompilationErrors.unsupportedTableOperationError(ident, "DROP CONSTRAINT") + + case a: AddCheckConstraint + if a.child.exists { + case _: LogicalRelation => true + case _: HiveTableRelation => true + case _ => false + } => + val tableIdent = a.child.collectFirst { + case l: LogicalRelation => l.catalogTable.get.identifier + case h: HiveTableRelation => h.tableMeta.identifier + }.get + throw QueryCompilationErrors.unsupportedTableOperationError(tableIdent, "ADD CONSTRAINT") + case SetTableProperties(ResolvedV1TableIdentifier(ident), props) => AlterTableSetPropertiesCommand(ident, props, isView = false) @@ -187,6 +206,10 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) c.tableSpec.provider, tableSpec.options, c.tableSpec.location, c.tableSpec.serde, ctas = false) if (!isV2Provider(provider)) { + if (tableSpec.constraints.nonEmpty) { + throw QueryCompilationErrors.unsupportedTableOperationError( + ident, "CONSTRAINT") + } constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, c.ignoreIfExists, storageFormat, provider) } else { @@ -203,6 +226,10 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) ctas = true) if (!isV2Provider(provider)) { + if (tableSpec.constraints.nonEmpty) { + throw QueryCompilationErrors.unsupportedTableOperationError( + ident, "CONSTRAINT") + } constructV1TableCmd(Some(c.query), c.tableSpec, ident, new StructType, c.partitioning, c.ignoreIfExists, storageFormat, provider) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TableConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TableConstraintSuite.scala new file mode 100644 index 0000000000000..f8b3411d59a05 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TableConstraintSuite.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.execution.command.DDLCommandTestUtils + +/** + * This base suite contains unified tests for table constraints (CHECK, PRIMARY KEY, UNIQUE, + * FOREIGN KEY) that check V1 table catalogs. V1 tables do not support table constraints. + * The tests that cannot run for all V1 catalogs are located in more specific test suites: + * + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.TableConstraintSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.TableConstraintSuite` + */ +trait TableConstraintSuiteBase extends DDLCommandTestUtils { + override val command = "TABLE CONSTRAINT" + + private val constraintTypes = Seq( + "CHECK (id > 0)", + "PRIMARY KEY (id)", + "UNIQUE (id)", + "FOREIGN KEY (id) REFERENCES t2(id)" + ) + + gridTest("SPARK-54761: create table with constraint - should fail")(constraintTypes) + { constraint => + withNamespaceAndTable("ns", "table_1") { t => + val createTableSql = s"CREATE TABLE $t (id INT, CONSTRAINT c1 $constraint) $defaultUsing" + val error = intercept[AnalysisException] { + sql(createTableSql) + } + checkError( + exception = error, + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> s"`$catalog`.`ns`.`table_1`", + "operation" -> "CONSTRAINT" + ) + ) + } + } + + gridTest("SPARK-54761: alter table add constraint - should fail")(constraintTypes) { constraint => + withNamespaceAndTable("ns", "table_1") { t => + sql(s"CREATE TABLE $t (id INT) $defaultUsing") + val alterTableSql = s"ALTER TABLE $t ADD CONSTRAINT c1 $constraint" + val error = intercept[AnalysisException] { + sql(alterTableSql) + } + checkError( + exception = error, + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> s"`$catalog`.`ns`.`table_1`", + "operation" -> "ADD CONSTRAINT" + ) + ) + } + } + + test("SPARK-54761: alter table drop constraint - should fail") { + withNamespaceAndTable("ns", "table_1") { t => + sql(s"CREATE TABLE $t (id INT) $defaultUsing") + val error = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP CONSTRAINT c1") + } + checkError( + exception = error, + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> s"`$catalog`.`ns`.`table_1`", + "operation" -> "DROP CONSTRAINT" + ) + ) + } + } + + // REPLACE TABLE is not supported for V1 tables, so the error should be about + // REPLACE TABLE, not about CONSTRAINT + gridTest("SPARK-54761: replace table with constraint - should fail")(constraintTypes) + { constraint => + withNamespaceAndTable("ns", "table_1") { t => + val replaceTableSql = s"REPLACE TABLE $t (id INT, CONSTRAINT c1 $constraint) $defaultUsing" + val error = intercept[AnalysisException] { + sql(replaceTableSql) + } + checkError( + exception = error, + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> s"`$catalog`.`ns`.`table_1`", + "operation" -> "REPLACE TABLE" + ) + ) + } + } +} + +/** + * The class contains tests for table constraints to check V1 In-Memory table catalog. + */ +class TableConstraintSuite extends TableConstraintSuiteBase with CommandSuiteBase + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index 44c5b1ad28745..ee2dd476958e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.catalyst.util.AttributeNameParser +import org.apache.spark.sql.catalyst.util.QuotingUtils.quoteNameParts import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.catalog.constraints.Check import org.apache.spark.sql.execution.command.DDLCommandTestUtils @@ -29,11 +31,11 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma override protected def command: String = "Check CONSTRAINT" test("Nondeterministic expression -- alter table") { - withTable("t") { - sql("create table t(i double)") + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + sql(s"CREATE TABLE $t (i DOUBLE) $defaultUsing") val query = - """ - |ALTER TABLE t ADD CONSTRAINT c1 CHECK (i > rand(0)) + s""" + |ALTER TABLE $t ADD CONSTRAINT c1 CHECK (i > rand(0)) |""".stripMargin val error = intercept[AnalysisException] { sql(query) @@ -45,8 +47,8 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma parameters = Map("checkCondition" -> "i > rand(0)"), context = ExpectedContext( fragment = "i > rand(0)", - start = 40, - stop = 50 + start = 67, + stop = 77 ) ) } @@ -77,27 +79,31 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } test("Expression referring a column of another table -- alter table") { - withTable("t", "t2") { - sql("CREATE TABLE t(i DOUBLE) USING parquet") - sql("CREATE TABLE t2(j STRING) USING parquet") - val query = - """ - |ALTER TABLE t ADD CONSTRAINT c1 CHECK (len(t2.j) > 0) - |""".stripMargin - val error = intercept[AnalysisException] { - sql(query) - } - checkError( - exception = error, - condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", - sqlState = "42703", - parameters = Map("objectName" -> "`t2`.`j`", "proposal" -> "`t`.`i`"), - context = ExpectedContext( - fragment = "t2.j", - start = 44, - stop = 47 + withNamespaceAndTable("ns", "tbl_1", nonPartitionCatalog) { t1 => + withNamespaceAndTable("ns", "tbl_2", nonPartitionCatalog) { t2 => + sql(s"CREATE TABLE $t1(i DOUBLE) $defaultUsing") + sql(s"CREATE TABLE $t2(j STRING) $defaultUsing") + val query = + s""" + |ALTER TABLE $t1 ADD CONSTRAINT c1 CHECK (len($t2.j) > 0) + |""".stripMargin + val error = intercept[AnalysisException] { + sql(query) + } + checkError( + exception = error, + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + sqlState = "42703", + parameters = Map( + "objectName" -> quoteNameParts(AttributeNameParser.parseAttributeName(s"$t2.j")), + "proposal" -> quoteNameParts(AttributeNameParser.parseAttributeName(s"$t1.i"))), + context = ExpectedContext( + fragment = s"$t2.j", + start = 73, + stop = 104 + ) ) - ) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index dd84e4d1420e9..847a956c11ab5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -217,10 +217,10 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase test("desc table constraints") { withNamespaceAndTable("ns", "pk_table", nonPartitionCatalog) { tbl => - withTable("fk_table") { + withNamespaceAndTable("ns", "fk_table", nonPartitionCatalog) { fkTable => sql( s""" - |CREATE TABLE fk_table (id INT PRIMARY KEY) USING parquet + |CREATE TABLE $fkTable (id INT PRIMARY KEY) $defaultUsing """.stripMargin) sql( s""" @@ -230,7 +230,7 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase | b STRING, | c STRING, | PRIMARY KEY (id), - | CONSTRAINT fk_a FOREIGN KEY (a) REFERENCES fk_table(id) RELY, + | CONSTRAINT fk_a FOREIGN KEY (a) REFERENCES $fkTable(id) RELY, | CONSTRAINT uk_b UNIQUE (b), | CONSTRAINT uk_a_c UNIQUE (a, c), | CONSTRAINT c1 CHECK (c IS NOT NULL), @@ -243,7 +243,7 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase var expectedConstraintsDdl = Array( "# Constraints,,", "pk_table_pk,PRIMARY KEY (id) NOT ENFORCED,", - "fk_a,FOREIGN KEY (a) REFERENCES fk_table (id) NOT ENFORCED RELY,", + s"fk_a,FOREIGN KEY (a) REFERENCES $fkTable (id) NOT ENFORCED RELY,", "uk_b,UNIQUE (b) NOT ENFORCED,", "uk_a_c,UNIQUE (a, c) NOT ENFORCED,", "c1,CHECK (c IS NOT NULL) ENFORCED,", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateTableSuite.scala index 2e3929d906ce3..a9b33584efc40 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowCreateTableSuite.scala @@ -184,13 +184,13 @@ class ShowCreateTableSuite extends command.ShowCreateTableSuiteBase with Command test("show table constraints") { withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => - withTable("other_table") { + withNamespaceAndTable("ns", "other_table", nonPartitionCatalog) { otherTable => sql( s""" - |CREATE TABLE other_table ( + |CREATE TABLE $otherTable ( | id STRING PRIMARY KEY |) - |USING parquet + |$defaultUsing """.stripMargin) sql( s""" @@ -200,7 +200,7 @@ class ShowCreateTableSuite extends command.ShowCreateTableSuiteBase with Command | c STRING, | PRIMARY KEY (a), | CONSTRAINT uk_b UNIQUE (b), - | CONSTRAINT fk_c FOREIGN KEY (c) REFERENCES other_table(id) RELY, + | CONSTRAINT fk_c FOREIGN KEY (c) REFERENCES $otherTable(id) RELY, | CONSTRAINT c1 CHECK (c IS NOT NULL), | CONSTRAINT c2 CHECK (a > 0) |) @@ -214,7 +214,7 @@ class ShowCreateTableSuite extends command.ShowCreateTableSuiteBase with Command "c STRING,", "CONSTRAINT tbl_pk PRIMARY KEY (a) NOT ENFORCED NORELY,", "CONSTRAINT uk_b UNIQUE (b) NOT ENFORCED NORELY,", - "CONSTRAINT fk_c FOREIGN KEY (c) REFERENCES other_table (id) NOT ENFORCED RELY,", + s"CONSTRAINT fk_c FOREIGN KEY (c) REFERENCES $otherTable (id) NOT ENFORCED RELY,", "CONSTRAINT c1 CHECK (c IS NOT NULL) ENFORCED NORELY," ) assert(showDDL === expectedDDLPrefix ++ Array( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/TableConstraintSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/TableConstraintSuite.scala new file mode 100644 index 0000000000000..540b19e4577c9 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/TableConstraintSuite.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 + +/** + * The class contains tests for table constraints to check V1 Hive external table catalog. + */ +class TableConstraintSuite extends v1.TableConstraintSuiteBase with CommandSuiteBase + From bdc850f91e28044d668c146a9be76dac419a1e94 Mon Sep 17 00:00:00 2001 From: Kristin Cowalcijk Date: Sun, 21 Dec 2025 14:45:28 +0900 Subject: [PATCH 314/400] [SPARK-54745][PYTHON] Fix PySpark import error caused by missing UnixStreamServer on Windows ### What changes were proposed in this pull request? This PR fixes an error caused by `socketserver.UnixStreamServer` not being available on Windows. We define a fallback `AccumulatorUnixServer` to raise an exception on construction and inform the user to disable `spark.python.unix.domain.socket.enabled`. ### Why are the changes needed? `import pyspark` fails with the following message on Windows since PySpark 4.1.0: ``` sedona\spark\__init__.py:19: in import pyspark .venv\Lib\site-packages\pyspark\__init__.py:71: in from pyspark.accumulators import Accumulator, AccumulatorParam .venv\Lib\site-packages\pyspark\accumulators.py:324: in class AccumulatorUnixServer(socketserver.UnixStreamServer): ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ E AttributeError: module 'socketserver' has no attribute 'UnixStreamServer' ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually test this on Windows 11 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53546 from Kontinuation/fix-uds-windows-compat. Authored-by: Kristin Cowalcijk Signed-off-by: Hyukjin Kwon (cherry picked from commit a8f817d1ad9df6bb2eac7a3b6645e55a4379a6c6) Signed-off-by: Hyukjin Kwon --- python/pyspark/accumulators.py | 45 ++++++++++++++++++++++------------ 1 file changed, 30 insertions(+), 15 deletions(-) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 59f7856688ee9..e557fe1cd8fb8 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -321,21 +321,36 @@ def shutdown(self) -> None: self.server_close() -class AccumulatorUnixServer(socketserver.UnixStreamServer): - server_shutdown = False - - def __init__( - self, socket_path: str, RequestHandlerClass: Type[socketserver.BaseRequestHandler] - ): - super().__init__(socket_path, RequestHandlerClass) - self.auth_token = None - - def shutdown(self) -> None: - self.server_shutdown = True - super().shutdown() - self.server_close() - if os.path.exists(self.server_address): # type: ignore[arg-type] - os.remove(self.server_address) # type: ignore[arg-type] +# socketserver.UnixStreamServer is not available on Windows yet +# (https://github.com/python/cpython/issues/77589). +if hasattr(socketserver, "UnixStreamServer"): + + class AccumulatorUnixServer(socketserver.UnixStreamServer): + server_shutdown = False + + def __init__( + self, socket_path: str, RequestHandlerClass: Type[socketserver.BaseRequestHandler] + ): + super().__init__(socket_path, RequestHandlerClass) + self.auth_token = None + + def shutdown(self) -> None: + self.server_shutdown = True + super().shutdown() + self.server_close() + if os.path.exists(self.server_address): # type: ignore[arg-type] + os.remove(self.server_address) # type: ignore[arg-type] + +else: + + class AccumulatorUnixServer(socketserver.TCPServer): # type: ignore[no-redef] + def __init__( + self, socket_path: str, RequestHandlerClass: Type[socketserver.BaseRequestHandler] + ): + raise NotImplementedError( + "Unix Domain Sockets are not supported on this platform. " + "Please disable it by setting spark.python.unix.domain.socket.enabled to false." + ) def _start_update_server( From c76797925eaec6e1904e3d44a42b50a838f1b6fe Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 22 Dec 2025 07:51:48 +0800 Subject: [PATCH 315/400] [SPARK-54782][SQL] Correct the config versions ### What changes were proposed in this pull request? Correct the config versions ### Why are the changes needed? the config versions are incorrect ### Does this PR introduce _any_ user-facing change? doc-only change ### How was this patch tested? manually check ### Was this patch authored or co-authored using generative AI tooling? no Closes #53545 from zhengruifeng/correct_version_info. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit d0cbad56a10502a1c931d5967beeae2369f6fa15) Signed-off-by: Ruifeng Zheng --- .../apache/spark/sql/internal/SQLConf.scala | 64 +++++++++++-------- 1 file changed, 36 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 36febe67f4096..994291d3db5a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -243,40 +243,43 @@ object SQLConf { val PREFER_COLUMN_OVER_LCA_IN_ARRAY_INDEX = buildConf("spark.sql.analyzer.preferColumnOverLcaInArrayIndex") - .internal() - .doc( - "When true, prefer the column from the underlying relation over the lateral column alias " + - "reference with the same name (see SPARK-53734)." - ) - .booleanConf - .createWithDefault(true) + .internal() + .version("4.1.0") + .doc( + "When true, prefer the column from the underlying relation over the lateral column alias " + + "reference with the same name (see SPARK-53734).") + .booleanConf + .createWithDefault(true) val DONT_DEDUPLICATE_EXPRESSION_IF_EXPR_ID_IN_OUTPUT = buildConf("spark.sql.analyzer.dontDeduplicateExpressionIfExprIdInOutput") - .internal() - .doc( - "DeduplicateRelations shouldn't remap expressions to new ExprIds if old ExprId still " + - "exists in output.") - .booleanConf - .createWithDefault(true) + .internal() + .version("4.1.0") + .doc( + "DeduplicateRelations shouldn't remap expressions to new ExprIds if old ExprId still " + + "exists in output.") + .booleanConf + .createWithDefault(true) val UNION_IS_RESOLVED_WHEN_DUPLICATES_PER_CHILD_RESOLVED = buildConf("spark.sql.analyzer.unionIsResolvedWhenDuplicatesPerChildResolved") - .internal() - .doc( - "When true, union should only be resolved once there are no duplicate attributes in " + - "each branch.") - .booleanConf - .createWithDefault(true) + .internal() + .version("4.1.0") + .doc( + "When true, union should only be resolved once there are no duplicate attributes in " + + "each branch.") + .booleanConf + .createWithDefault(true) val ONLY_NECESSARY_AND_UNIQUE_METADATA_COLUMNS = buildConf("spark.sql.analyzer.uniqueNecessaryMetadataColumns") - .internal() - .doc( - "When this conf is enabled, AddMetadataColumns rule should only add necessary metadata " + - "columns and only if those columns are not already present in the project list.") - .booleanConf - .createWithDefault(true) + .internal() + .version("4.1.0") + .doc( + "When this conf is enabled, AddMetadataColumns rule should only add necessary metadata " + + "columns and only if those columns are not already present in the project list.") + .booleanConf + .createWithDefault(true) val BLOCK_CREATE_TEMP_TABLE_USING_PROVIDER = buildConf("spark.sql.legacy.blockCreateTempTableUsingProvider") @@ -324,7 +327,7 @@ object SQLConf { "(AliasResolution.resolve, FunctionResolution.resolveFunction, etc)." + "This feature is currently under development." ) - .version("4.0.0") + .version("4.1.0") .booleanConf .createWithDefault(false) @@ -1057,7 +1060,7 @@ object SQLConf { "An object with an explicitly set collation will not inherit the collation from the " + "schema." ) - .version("4.0.0") + .version("4.1.0") .booleanConf .createWithDefault(false) @@ -1911,6 +1914,7 @@ object SQLConf { val DATA_SOURCE_V2_JOIN_PUSHDOWN = buildConf("spark.sql.optimizer.datasourceV2JoinPushdown") .internal() + .version("4.1.0") .doc("When this config is set to true, join is tried to be pushed down" + "for DSv2 data sources in V2ScanRelationPushdown optimization rule.") .booleanConf @@ -1919,6 +1923,7 @@ object SQLConf { val DATA_SOURCE_V2_EXPR_FOLDING = buildConf("spark.sql.optimizer.datasourceV2ExprFolding") .internal() + .version("4.1.0") .doc("When this config is set to true, do safe constant folding for the " + "expressions before translation and pushdown.") .booleanConf @@ -2549,6 +2554,7 @@ object SQLConf { val STATE_STORE_MAINTENANCE_SHUTDOWN_TIMEOUT = buildConf("spark.sql.streaming.stateStore.maintenanceShutdownTimeout") .internal() + .version("4.1.0") .doc("Timeout in seconds for maintenance pool operations to complete on shutdown") .timeConf(TimeUnit.SECONDS) .createWithDefault(300L) @@ -2556,6 +2562,7 @@ object SQLConf { val STATE_STORE_MAINTENANCE_PROCESSING_TIMEOUT = buildConf("spark.sql.streaming.stateStore.maintenanceProcessingTimeout") .internal() + .version("4.1.0") .doc("Timeout in seconds to wait for maintenance to process this partition.") .timeConf(TimeUnit.SECONDS) .createWithDefault(30L) @@ -2674,6 +2681,7 @@ object SQLConf { "Note: For structured streaming, this configuration cannot be changed between query " + "restarts from the same checkpoint location.") .internal() + .version("4.1.0") .intConf .checkValue(_ > 0, "The value of spark.sql.streaming.internal.stateStore.partitions must be a positive " + @@ -3483,7 +3491,7 @@ object SQLConf { .doc("When true, if a microbatch is retried, if a file already exists but its checksum " + "file does not exist, the file checksum will not be created. This is useful for " + "compatibility with files created before file checksums were enabled.") - .version("4.2.0") + .version("4.1.0") .booleanConf .createWithDefault(true) From 8f544a50359b837fd536e4170edd109ac99dad7b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 22 Dec 2025 12:05:52 +0900 Subject: [PATCH 316/400] [SPARK-54794][CORE] Suppress verbose `FsHistoryProvider.checkForLogs` scanning logs ### What changes were proposed in this pull request? This PR aims to suppress verbose `FsHistoryProvider.checkForLogs` scanning logs. ### Why are the changes needed? This is a kind of regression at Apache Spark 4.1.0 switched from `logDebug` to `logInfo` accidentally. We had better recover them back to `DEBUG` level because it's excessively verbose currently. - #52382 ``` 25/12/21 22:27:53 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356063285 25/12/21 22:28:03 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356073312 25/12/21 22:28:13 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356083338 25/12/21 22:28:23 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356093347 25/12/21 22:28:33 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356103369 25/12/21 22:28:43 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356113391 25/12/21 22:28:53 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356123402 25/12/21 22:29:03 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356133422 25/12/21 22:29:13 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356143443 25/12/21 22:29:23 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356153458 25/12/21 22:29:33 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356163471 25/12/21 22:29:43 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356173491 25/12/21 22:29:53 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356183502 25/12/21 22:30:03 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356193517 25/12/21 22:30:13 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356203529 25/12/21 22:30:23 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356213555 25/12/21 22:30:33 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356223567 25/12/21 22:30:43 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356233585 25/12/21 22:30:53 INFO FsHistoryProvider: Scanning s3a://spark-events with lastScanTime=1766356243599 ``` ### Does this PR introduce _any_ user-facing change? No behavior change except log message level. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53555 from dongjoon-hyun/SPARK-54794. Authored-by: Dongjoon Hyun Signed-off-by: Kousuke Saruta (cherry picked from commit d0fbb15bbc98798473ba842dcfc2dec601a7a376) Signed-off-by: Kousuke Saruta --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 4863291b529bb..c723a8de8c442 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -502,7 +502,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) var count: Int = 0 try { val newLastScanTime = clock.getTimeMillis() - logInfo(log"Scanning ${MDC(HISTORY_DIR, logDir)} with " + + logDebug(log"Scanning ${MDC(HISTORY_DIR, logDir)} with " + log"lastScanTime=${MDC(LAST_SCAN_TIME, lastScanTime)}") // Mark entries that are processing as not stale. Such entries do not have a chance to be From af42e1b6af6d34b40d203aef9395acdc438cdf69 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 22 Dec 2025 13:56:29 +0800 Subject: [PATCH 317/400] [SPARK-53991][SQL][TEST][FOLLOWUP] Make KLL quantile golden file tests deterministic ### What changes were proposed in this pull request? In https://github.com/apache/spark/pull/52800, we added SQL support for KLL quantiles functions based on DataSketches. In this PR, we update some of the golden file tests to make them deterministic. ### Why are the changes needed? The previous tests generated string summaries of KLL quantile sketches and then split them by newlines and made case-sensitive checks for substrings. It turns out this was brittle, so this PR updates the tests to avoid the newline-splitting and makes the substring checks case-insenstiive. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? This PR updates test coverage only. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53549 from dtenedor/kll-quantile-golden-files-fix. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan (cherry picked from commit e4b99932d02cbc1c177e59f1da987856ece12f3d) Signed-off-by: Wenchen Fan --- .../analyzer-results/kllquantiles.sql.out | 57 ++++++++----------- .../sql-tests/inputs/kllquantiles.sql | 22 +++---- .../sql-tests/results/kllquantiles.sql.out | 34 +++++------ 3 files changed, 53 insertions(+), 60 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out index dc22199985f0b..3eea568420c04 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/kllquantiles.sql.out @@ -128,7 +128,7 @@ CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t_double_1_5_thr -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -136,8 +136,7 @@ FROM ( FROM t_byte_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_bigint(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_bigint(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_byte_1_5_through_7_11 @@ -145,7 +144,7 @@ Project [split(kll_sketch_to_string_bigint(agg#x), -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -153,8 +152,7 @@ FROM ( FROM t_int_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_bigint(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_bigint(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_int_1_5_through_7_11 @@ -162,7 +160,7 @@ Project [split(kll_sketch_to_string_bigint(agg#x), -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -170,8 +168,7 @@ FROM ( FROM t_long_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_bigint(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_bigint(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 @@ -179,7 +176,7 @@ Project [split(kll_sketch_to_string_bigint(agg#x), -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -187,8 +184,7 @@ FROM ( FROM t_short_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_bigint(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_bigint(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_bigint(agg#x, cast(0.5 as double)) - cast(4 as bigint))) < cast(1 as bigint)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_bigint(agg#x, cast(3 as bigint)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_bigint(col1#x, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_short_1_5_through_7_11 @@ -196,7 +192,7 @@ Project [split(kll_sketch_to_string_bigint(agg#x), -- !query -SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -204,8 +200,7 @@ FROM ( FROM t_float_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_float(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((cast(kll_sketch_get_quantile_float(agg#x, cast(0.5 as double)) as double) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_float(agg#x, cast(3 as float)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_float(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((cast(kll_sketch_get_quantile_float(agg#x, cast(0.5 as double)) as double) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_float(agg#x, cast(3 as float)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_float(col1#x, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 @@ -213,7 +208,7 @@ Project [split(kll_sketch_to_string_float(agg#x), -- !query -SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -221,8 +216,7 @@ FROM ( FROM t_double_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_double(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_double(agg#x, cast(3 as double)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_double(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_double(agg#x, cast(3 as double)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_double_1_5_through_7_11 @@ -230,7 +224,7 @@ Project [split(kll_sketch_to_string_double(agg#x), -- !query -SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -238,8 +232,7 @@ FROM ( FROM t_float_1_5_through_7_11 ) -- !query analysis -Project [split(kll_sketch_to_string_double(agg#x), -, -1)[1] LIKE %Kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_double(agg#x, cast(3 as double)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +Project [lower(kll_sketch_to_string_double(agg#x)) LIKE %kll% AS str_contains_kll#x, (abs((kll_sketch_get_quantile_double(agg#x, cast(0.5 as double)) - cast(4.0 as double))) < cast(0.5 as double)) AS median_close_to_4#x, (abs((kll_sketch_get_rank_double(agg#x, cast(3 as double)) - cast(0.4 as double))) < cast(0.1 as double)) AS rank3_close_to_0_4#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_double(col1#x, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_float_1_5_through_7_11 @@ -549,7 +542,7 @@ Aggregate [kll_sketch_get_n_bigint(kll_sketch_agg_bigint(col1#xL, Some(100), 0, -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -571,15 +564,15 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 273, - "stopIndex" : 299, + "startIndex" : 264, + "stopIndex" : 290, "fragment" : "kll_sketch_agg_bigint(col1)" } ] } -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -601,15 +594,15 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 273, - "stopIndex" : 299, + "startIndex" : 264, + "stopIndex" : 290, "fragment" : "kll_sketch_agg_bigint(col1)" } ] } -- !query -SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -631,8 +624,8 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 274, - "stopIndex" : 299, + "startIndex" : 265, + "stopIndex" : 290, "fragment" : "kll_sketch_agg_float(col1)" } ] } @@ -931,13 +924,13 @@ Project [isnotnull(kll_sketch_get_quantile_float(agg#x, cast(0.5 as double))) AS -- !query -SELECT kll_sketch_to_string_double(agg) LIKE '%Kll%' AS contains_kll_header +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS contains_kll_header FROM ( SELECT kll_sketch_agg_bigint(col1) AS agg FROM t_long_1_5_through_7_11 ) -- !query analysis -Project [kll_sketch_to_string_double(agg#x) LIKE %Kll% AS contains_kll_header#x] +Project [lower(kll_sketch_to_string_double(agg#x)) LIKE %kll% AS contains_kll_header#x] +- SubqueryAlias __auto_generated_subquery_name +- Aggregate [kll_sketch_agg_bigint(col1#xL, None, 0, 0) AS agg#x] +- SubqueryAlias spark_catalog.default.t_long_1_5_through_7_11 diff --git a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql index 69d472ac78a60..fe1b61de037d7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/kllquantiles.sql @@ -64,7 +64,7 @@ VALUES (CAST(7 AS DOUBLE), CAST(11 AS DOUBLE)) AS tab(col1, col2); -- BIGINT sketches -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -72,7 +72,7 @@ FROM ( FROM t_byte_1_5_through_7_11 ); -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -80,7 +80,7 @@ FROM ( FROM t_int_1_5_through_7_11 ); -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -88,7 +88,7 @@ FROM ( FROM t_long_1_5_through_7_11 ); -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -97,7 +97,7 @@ FROM ( ); -- FLOAT sketches (only accepts float types to avoid precision loss) -SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -106,7 +106,7 @@ FROM ( ); -- DOUBLE sketches (accepts float and double types to avoid precision loss from integer conversion) -SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -115,7 +115,7 @@ FROM ( ); -- Test float column with double sketch (valid type promotion) -SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -268,7 +268,7 @@ FROM t_long_1_5_through_7_11; -- These queries should fail with type mismatch or validation errors -- Type mismatch: BIGINT sketch does not accept DOUBLE columns -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -277,7 +277,7 @@ FROM ( ); -- Type mismatch: BIGINT sketch does not accept FLOAT columns -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -286,7 +286,7 @@ FROM ( ); -- Type mismatch: FLOAT sketch does not accept DOUBLE columns -SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -378,7 +378,7 @@ FROM ( -- interpret the binary data. This query succeeds even though we're using a DOUBLE -- to_string function on a BIGINT sketch. The function reads the binary representation -- and produces output, but the numeric values will be incorrectly interpreted. -SELECT kll_sketch_to_string_double(agg) LIKE '%Kll%' AS contains_kll_header +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS contains_kll_header FROM ( SELECT kll_sketch_agg_bigint(col1) AS agg FROM t_long_1_5_through_7_11 diff --git a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out index 6f60f30e5681e..863bded1599b5 100644 --- a/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/kllquantiles.sql.out @@ -134,7 +134,7 @@ struct<> -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -148,7 +148,7 @@ true true true -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -162,7 +162,7 @@ true true true -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -176,7 +176,7 @@ true true true -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -190,7 +190,7 @@ true true true -- !query -SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -204,7 +204,7 @@ true true true -- !query -SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -218,7 +218,7 @@ true true true -- !query -SELECT split(kll_sketch_to_string_double(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_double(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_double(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -512,7 +512,7 @@ struct -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -536,15 +536,15 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 273, - "stopIndex" : 299, + "startIndex" : 264, + "stopIndex" : 290, "fragment" : "kll_sketch_agg_bigint(col1)" } ] } -- !query -SELECT split(kll_sketch_to_string_bigint(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_bigint(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_bigint(agg, 0.5) - 4) < 1 AS median_close_to_4, abs(kll_sketch_get_rank_bigint(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -568,15 +568,15 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 273, - "stopIndex" : 299, + "startIndex" : 264, + "stopIndex" : 290, "fragment" : "kll_sketch_agg_bigint(col1)" } ] } -- !query -SELECT split(kll_sketch_to_string_float(agg), '\n')[1] LIKE '%Kll%' AS str_contains_kll, +SELECT lower(kll_sketch_to_string_float(agg)) LIKE '%kll%' AS str_contains_kll, abs(kll_sketch_get_quantile_float(agg, 0.5) - 4.0) < 0.5 AS median_close_to_4, abs(kll_sketch_get_rank_float(agg, 3) - 0.4) < 0.1 AS rank3_close_to_0_4 FROM ( @@ -600,8 +600,8 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 274, - "stopIndex" : 299, + "startIndex" : 265, + "stopIndex" : 290, "fragment" : "kll_sketch_agg_float(col1)" } ] } @@ -953,7 +953,7 @@ true -- !query -SELECT kll_sketch_to_string_double(agg) LIKE '%Kll%' AS contains_kll_header +SELECT lower(kll_sketch_to_string_double(agg)) LIKE '%kll%' AS contains_kll_header FROM ( SELECT kll_sketch_agg_bigint(col1) AS agg FROM t_long_1_5_through_7_11 From 8352e3a0a301369b52af7beafcb1fd2f5645668b Mon Sep 17 00:00:00 2001 From: Chris Boumalhab Date: Mon, 22 Dec 2025 14:27:40 +0800 Subject: [PATCH 318/400] [SPARK-52407][SQL][FOLLOWUP] Expression description fix for ThetaIntersectionAgg ### What changes were proposed in this pull request? This PR corrects the ExpressionDescription usage text for theta_intersection_agg function. The usage description was updated from: `_FUNC_(expr, lgNomEntries)` to: `_FUNC_(expr)` ### Why are the changes needed? The previous documentation incorrectly showed lgNomEntries as a parameter for theta_intersection_agg. This function only accepts a single expression parameter, unlike theta_sketch_agg and theta_union_agg which both accept an optional lgNomEntries parameter. This change ensures the documentation accurately reflects the function's signature and provides a clearer description of what the function does. ### Does this PR introduce _any_ user-facing change? Yes - the documentation shown in SQL help/describe function output for theta_intersection_agg will now correctly show only one parameter instead of two, and will include a more descriptive explanation of the function's behavior. ### How was this patch tested? The change is documentation-only. The function signature and implementation remain unchanged. Existing tests for theta_intersection_agg continue to validate the correct behavior. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53552 from cboumalh/cboumalh-theta-followup. Authored-by: Chris Boumalhab Signed-off-by: Wenchen Fan (cherry picked from commit e4b75089034b7a735f0f6baafe26d4f6a782a52a) Signed-off-by: Wenchen Fan --- .../expressions/aggregate/thetasketchesAggregates.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala index f841632b14622..a14df39bf822d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/thetasketchesAggregates.scala @@ -497,7 +497,7 @@ case class ThetaUnionAgg( // scalastyle:off line.size.limit @ExpressionDescription( usage = """ - _FUNC_(expr, lgNomEntries) - Returns the ThetaSketch's Compact binary representation + _FUNC_(expr) - Returns the ThetaSketch's Compact binary representation by intersecting all the Theta sketches in the input column.""", examples = """ Examples: From d1b7386e267e6d06e96612f9d663c971b696ff06 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stefan=20Savi=C4=87?= Date: Tue, 23 Dec 2025 00:07:20 +0800 Subject: [PATCH 319/400] [SPARK-54800] Changed default implementation for isObjectNotFoundException MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Changed that `isObjectNotFoundException`default implementation considers an exception when its `SQLState` starts with 42. ### Why are the changes needed? Before, if `isObjectNotFound` was not overridden (for example, in a custom Dialect), then when any other error was thrown that was not ‘table does not exist’, it would still be reported as a TableNotFound error, which is incorrect ### Does this PR introduce _any_ user-facing change? Updated method in JDBCDialects. ### How was this patch tested? In another PR #53541. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53456 from SavicStefan/stefan-savic_data/stefan. Authored-by: Stefan Savić Signed-off-by: Wenchen Fan (cherry picked from commit aad5b6e19c6eb9280ae7e1f04cd39b2f055aa3ce) Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index ce4c347cad349..6b4e743ce989c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -772,7 +772,9 @@ abstract class JdbcDialect extends Serializable with Logging { } @Since("4.1.0") - def isObjectNotFoundException(e: SQLException): Boolean = true + def isObjectNotFoundException(e: SQLException): Boolean = { + Option(e.getSQLState).exists(_.startsWith("42")) + } /** * Gets a dialect exception, classifies it and wraps it by `AnalysisException`. From 3eed713e51a9d485c9bc5cffb82f4f2ee58e92b4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 23 Dec 2025 09:21:03 +0900 Subject: [PATCH 320/400] [SPARK-54801][SQL] Mark a few new 4.1 configs as internal ### What changes were proposed in this pull request? We don't expect users to specify these configs as they touch internal details. For example, recursive CTE can use SQL syntax `MAX RECURSION LEVEL` and `LIMIT` to specify the row/level limits, people shouldn't set the configs. ### Why are the changes needed? avoid exposing internal configs. ### Does this PR introduce _any_ user-facing change? no, it's actually a doc change. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? cursor 2.2.20 Closes #53563 from cloud-fan/internal_conf. Authored-by: Wenchen Fan Signed-off-by: Hyukjin Kwon (cherry picked from commit c9cb5a240144b1ef6ce272850834250d14eb9fa3) Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 994291d3db5a9..100149a39211f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -575,6 +575,7 @@ object SQLConf { val GEOSPATIAL_ENABLED = buildConf("spark.sql.geospatial.enabled") + .internal() .doc("When true, enables geospatial types (GEOGRAPHY/GEOMETRY) and ST functions.") .version("4.1.0") .booleanConf @@ -934,6 +935,7 @@ object SQLConf { val ADAPTIVE_EXECUTION_ENABLED_IN_STATELESS_STREAMING = buildConf("spark.sql.adaptive.streaming.stateless.enabled") + .internal() .doc("When true, enable adaptive query execution for stateless streaming query. To " + "enable this config, `spark.sql.adaptive.enabled` needs to be also enabled.") .version("4.1.0") @@ -1221,6 +1223,7 @@ object SQLConf { val MAP_ZIP_WITH_USES_JAVA_COLLECTIONS = buildConf("spark.sql.mapZipWithUsesJavaCollections") + .internal() .doc("When true, the `map_zip_with` function uses Java collections instead of Scala " + "collections. This is useful for avoiding NaN equality issues.") .version("4.1.0") @@ -1564,6 +1567,7 @@ object SQLConf { val PARQUET_VECTORIZED_READER_NULL_TYPE_ENABLED = buildConf("spark.sql.parquet.enableNullTypeVectorizedReader") + .internal() .doc("Enables vectorized Parquet reader support for NullType columns.") .version("4.1.0") .booleanConf @@ -1597,6 +1601,7 @@ object SQLConf { val PARQUET_ANNOTATE_VARIANT_LOGICAL_TYPE = buildConf("spark.sql.parquet.variant.annotateLogicalType.enabled") + .internal() .doc("When enabled, Spark annotates the variant groups written to Parquet as the parquet " + "variant logical type.") .version("4.1.0") @@ -5280,6 +5285,7 @@ object SQLConf { .createWithDefault(LegacyBehaviorPolicy.CORRECTED) val CTE_RECURSION_LEVEL_LIMIT = buildConf("spark.sql.cteRecursionLevelLimit") + .internal() .doc("Maximum level of recursion that is allowed while executing a recursive CTE definition." + "If a query does not get exhausted before reaching this limit it fails. Use -1 for " + "unlimited.") @@ -5288,6 +5294,7 @@ object SQLConf { .createWithDefault(100) val CTE_RECURSION_ROW_LIMIT = buildConf("spark.sql.cteRecursionRowLimit") + .internal() .doc("Maximum number of rows that can be returned when executing a recursive CTE definition." + "If a query does not get exhausted before reaching this limit it fails. Use -1 for " + "unlimited.") @@ -5297,6 +5304,7 @@ object SQLConf { val CTE_RECURSION_ANCHOR_ROWS_LIMIT_TO_CONVERT_TO_LOCAL_RELATION = buildConf("spark.sql.cteRecursionAnchorRowsLimitToConvertToLocalRelation") + .internal() .doc("Maximum number of rows that the anchor in a recursive CTE can return for it to be" + "converted to a localRelation.") .version("4.1.0") @@ -5458,6 +5466,7 @@ object SQLConf { .createWithDefault(false) val PYTHON_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.python.filterPushdown.enabled") + .internal() .doc("When true, enable filter pushdown to Python datasource, at the cost of running " + "Python worker one additional time during planning.") .version("4.1.0") @@ -6631,6 +6640,7 @@ object SQLConf { val TIME_TYPE_ENABLED = buildConf("spark.sql.timeType.enabled") + .internal() .doc("When true, the TIME data type is supported.") .version("4.1.0") .booleanConf From 3e4b1cee13ad644a28dbc6ecff40e8b8358b8b74 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Tue, 23 Dec 2025 13:18:59 +0800 Subject: [PATCH 321/400] [SPARK-46741][SQL] Cache Table with CTE should work when CTE in plan expression subquery ### What changes were proposed in this pull request? Follow comment https://github.com/apache/spark/pull/53333#discussion_r2629958838 ### Why are the changes needed? Support all case ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #53526 from AngersZhuuuu/SPARK-46741-FOLLOWUP. Lead-authored-by: Angerszhuuuu Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit d65ee811b0f3616ecbd49ce77c93217a4d47ca82) Signed-off-by: Wenchen Fan --- .../catalyst/normalizer/NormalizeCTEIds.scala | 29 ++++++++++++++----- .../apache/spark/sql/CachedTableSuite.scala | 27 +++++++++++++++++ 2 files changed, 48 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala index 1b1b526e78140..6c0bca0e1104f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/normalizer/NormalizeCTEIds.scala @@ -17,29 +17,42 @@ package org.apache.spark.sql.catalyst.normalizer +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable + import org.apache.spark.sql.catalyst.plans.logical.{CacheTableAsSelect, CTERelationRef, LogicalPlan, UnionLoop, UnionLoopRef, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule -object NormalizeCTEIds extends Rule[LogicalPlan]{ +object NormalizeCTEIds extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { val curId = new java.util.concurrent.atomic.AtomicLong() - plan transformDown { + val cteIdToNewId = mutable.Map.empty[Long, Long] + applyInternal(plan, curId, cteIdToNewId) + } + private def applyInternal( + plan: LogicalPlan, + curId: AtomicLong, + cteIdToNewId: mutable.Map[Long, Long]): LogicalPlan = { + plan transformDownWithSubqueries { case ctas @ CacheTableAsSelect(_, plan, _, _, _, _, _) => - ctas.copy(plan = apply(plan)) + ctas.copy(plan = applyInternal(plan, curId, cteIdToNewId)) case withCTE @ WithCTE(plan, cteDefs) => - val defIdToNewId = withCTE.cteDefs.map(_.id).map((_, curId.getAndIncrement())).toMap - val normalizedPlan = canonicalizeCTE(plan, defIdToNewId) val newCteDefs = cteDefs.map { cteDef => - val normalizedCteDef = canonicalizeCTE(cteDef.child, defIdToNewId) - cteDef.copy(child = normalizedCteDef, id = defIdToNewId(cteDef.id)) + cteIdToNewId.getOrElseUpdate(cteDef.id, curId.getAndIncrement()) + val normalizedCteDef = canonicalizeCTE(cteDef.child, cteIdToNewId) + cteDef.copy(child = normalizedCteDef, id = cteIdToNewId(cteDef.id)) } + val normalizedPlan = canonicalizeCTE(plan, cteIdToNewId) withCTE.copy(plan = normalizedPlan, cteDefs = newCteDefs) } } - def canonicalizeCTE(plan: LogicalPlan, defIdToNewId: Map[Long, Long]): LogicalPlan = { + private def canonicalizeCTE( + plan: LogicalPlan, + defIdToNewId: mutable.Map[Long, Long]): LogicalPlan = { plan.transformDownWithSubqueries { // For nested WithCTE, if defIndex didn't contain the cteId, // means it's not current WithCTE's ref. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 880d8d72c73e7..0d807aeae4d7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -2633,6 +2633,33 @@ class CachedTableSuite extends QueryTest with SQLTestUtils } assert(inMemoryTableScan.size == 1) checkAnswer(df, Row(5) :: Nil) + + sql( + """ + |CACHE TABLE cache_subquery_cte_table + |WITH v AS ( + | SELECT c1 * c2 c3 from t1 + |) + |SELECT * + |FROM v + |WHERE EXISTS ( + | WITH cte AS (SELECT 1 AS id) + | SELECT 1 + | FROM cte + | WHERE cte.id = v.c3 + |) + |""".stripMargin) + + val cteInSubquery = sql( + """ + |SELECT * FROM cache_subquery_cte_table + |""".stripMargin) + + val subqueryInMemoryTableScan = collect(cteInSubquery.queryExecution.executedPlan) { + case i: InMemoryTableScanExec => i + } + assert(subqueryInMemoryTableScan.size == 1) + checkAnswer(cteInSubquery, Row(1) :: Nil) } } From 02f34a3a5a74f05a5c15e259630b069fd260a0b8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 23 Dec 2025 15:46:18 +0800 Subject: [PATCH 322/400] [SPARK-54815][CONNECT] Do not close the class loader of the session state if session is still in use ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/53233 . When session state is evicted from `Executor#isolatedSessionCache`, the session may still being used by the running tasks. This PR adds ref counting and skips closing class loader if the session is still in use. ### Why are the changes needed? closing class loader may break running tasks ### Does this PR introduce _any_ user-facing change? Yes, previously long running tasks may fail, and now it's fixed. ### How was this patch tested? existing tests. It's hard to construct a long running task and test the class loader behavior, but this fix is quite obvious. ### Was this patch authored or co-authored using generative AI tooling? cursor 2.2.20 Closes #53569 from cloud-fan/cache. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit db36f74f8fb0e34e366daec6fad2510089be7dc2) Signed-off-by: Wenchen Fan --- .../org/apache/spark/executor/Executor.scala | 93 +++++++++++++------ 1 file changed, 66 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index aa17f954a7d96..edab354b96078 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -24,7 +24,7 @@ import java.net.{URI, URL, URLClassLoader} import java.nio.ByteBuffer import java.util.{Locale, Properties} import java.util.concurrent._ -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import java.util.concurrent.locks.ReentrantLock import javax.annotation.concurrent.GuardedBy @@ -59,13 +59,63 @@ import org.apache.spark.util._ import org.apache.spark.util.ArrayImplicits._ private[spark] class IsolatedSessionState( - val sessionUUID: String, - var urlClassLoader: MutableURLClassLoader, - var replClassLoader: ClassLoader, - val currentFiles: HashMap[String, Long], - val currentJars: HashMap[String, Long], - val currentArchives: HashMap[String, Long], - val replClassDirUri: Option[String]) + val sessionUUID: String, + var urlClassLoader: MutableURLClassLoader, + var replClassLoader: ClassLoader, + val currentFiles: HashMap[String, Long], + val currentJars: HashMap[String, Long], + val currentArchives: HashMap[String, Long], + val replClassDirUri: Option[String]) extends Logging { + + // Reference count for the number of running tasks using this session. + private val refCount: AtomicInteger = new AtomicInteger(0) + + // Whether this session has been evicted from the cache. + @volatile private var evicted: Boolean = false + + /** Increment the reference count, indicating a task is using this session. */ + def acquire(): Unit = refCount.incrementAndGet() + + /** Decrement the reference count. If evicted and no more tasks, clean up. */ + def release(): Unit = { + if (refCount.decrementAndGet() == 0 && evicted) { + cleanup() + } + } + + /** Mark this session as evicted. If no tasks are using it, clean up immediately. */ + def markEvicted(): Unit = { + evicted = true + if (refCount.get() == 0) { + cleanup() + } else { + logInfo(log"Session ${MDC(SESSION_ID, sessionUUID)} evicted but still in use by " + + log"${MDC(LogKeys.COUNT, refCount.get())} task(s), deferring cleanup") + } + } + + private def cleanup(): Unit = { + // Close the urlClassLoader to release resources. + try { + urlClassLoader match { + case cl: URLClassLoader => + cl.close() + logInfo(log"Closed urlClassLoader for session ${MDC(SESSION_ID, sessionUUID)}") + case _ => + } + } catch { + case NonFatal(e) => + logWarning(log"Failed to close urlClassLoader for session " + + log"${MDC(SESSION_ID, sessionUUID)}", e) + } + // Delete session files. + val sessionBasedRoot = new File(SparkFiles.getRootDirectory(), sessionUUID) + if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) { + Utils.deleteRecursively(sessionBasedRoot) + } + logInfo(log"Session cleaned up: ${MDC(SESSION_ID, sessionUUID)}") + } +} /** * Spark executor, backed by a threadpool to run tasks. @@ -220,25 +270,9 @@ private[spark] class Executor( val state = notification.getValue // Cache is always used for isolated sessions. assert(!isDefaultState(state.sessionUUID)) - // Close the urlClassLoader to release resources. - try { - state.urlClassLoader match { - case urlClassLoader: URLClassLoader => - urlClassLoader.close() - logInfo(log"Closed urlClassLoader (URLClassLoader) for evicted session " + - log"${MDC(SESSION_ID, state.sessionUUID)}") - case _ => - } - } catch { - case NonFatal(e) => - logWarning(log"Failed to close urlClassLoader for session " + - log"${MDC(SESSION_ID, state.sessionUUID)}", e) - } - val sessionBasedRoot = new File(SparkFiles.getRootDirectory(), state.sessionUUID) - if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) { - Utils.deleteRecursively(sessionBasedRoot) - } - logInfo(log"Session evicted: ${MDC(SESSION_ID, state.sessionUUID)}") + // Mark evicted - cleanup will happen immediately if no tasks are using it, + // or when the last task releases it. + state.markEvicted() } }) .build[String, IsolatedSessionState] @@ -600,6 +634,9 @@ private[spark] class Executor( case _ => defaultSessionState } + // Pin the session to prevent its class loader from being closed while this task is running. + isolatedSession.acquire() + setMDCForTask(taskName, mdcProperties) threadId = Thread.currentThread.getId Thread.currentThread.setName(threadName) @@ -905,6 +942,8 @@ private[spark] class Executor( // are known, and metricsPoller.onTaskStart was called. metricsPoller.onTaskCompletion(taskId, task.stageId, task.stageAttemptId) } + // Release the session reference. If evicted and this was the last task, cleanup happens. + isolatedSession.release() } } From 51042d66d8e88bdd1ee1a150b775d681e45d69d4 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 23 Dec 2025 15:56:14 +0800 Subject: [PATCH 323/400] [SPARK-54760][SQL] DelegatingCatalogExtension as session catalog supports both V1 and V2 functions ### What changes were proposed in this pull request? This PR fixes a bug that occurs when the user uses a custom `DelegatingCatalogExtension` as the session catalog, Spark can not load the v2 function properly provided by the catalog. A typical use case is Iceberg's `SparkSessionCatalog` ``` $ spark-sql \ --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ ... ``` ``` spark-sql (default)> SELECT spark_catalog.system.iceberg_version(); [ROUTINE_NOT_FOUND] The routine `system`.`iceberg_version` cannot be found. Verify the spelling and correctness of the schema and catalog. If you did not qualify the name with a schema and catalog, verify the current_schema() output, or qualify the name with the correct schema and catalog. To tolerate the error on drop use DROP ... IF EXISTS. SQLSTATE: 42883; line 1 pos 7 ``` ### Why are the changes needed? Fix bug. ### Does this PR introduce _any_ user-facing change? Yes, it fixes a bug. ### How was this patch tested? Add new UT. Also manually tested with Iceberg. ``` spark-sql (default)> SELECT spark_catalog.system.iceberg_version(); 1.10.0 Time taken: 1.715 seconds, Fetched 1 row(s) ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53531 from pan3793/SPARK-54760. Authored-by: Cheng Pan Signed-off-by: Wenchen Fan (cherry picked from commit 512099bfb77cbd98f5ea26fc37d06b694c11dee1) Signed-off-by: Wenchen Fan --- .../analysis/FunctionResolution.scala | 23 +-- .../identifier-clause-legacy.sql.out | 8 +- .../identifier-clause.sql.out | 8 +- .../results/identifier-clause-legacy.sql.out | 8 +- .../results/identifier-clause.sql.out | 8 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 8 +- .../connector/DataSourceV2FunctionSuite.scala | 182 +++++++++--------- .../DataSourceV2SQLSessionCatalogSuite.scala | 9 +- .../SupportsCatalogOptionsSuite.scala | 9 +- .../connector/TestV2SessionCatalogBase.scala | 60 +++++- 10 files changed, 197 insertions(+), 126 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala index 800126e0030e8..8d6e2931a73b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala @@ -26,17 +26,16 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{ CatalogManager, - CatalogV2Util, - FunctionCatalog, - Identifier, LookupCatalog } import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.functions.{ AggregateFunction => V2AggregateFunction, - ScalarFunction + ScalarFunction, + UnboundFunction } import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} +import org.apache.spark.sql.internal.connector.V1Function import org.apache.spark.sql.types._ class FunctionResolution( @@ -52,10 +51,14 @@ class FunctionResolution( resolveBuiltinOrTempFunction(u.nameParts, u.arguments, u).getOrElse { val CatalogAndIdentifier(catalog, ident) = relationResolution.expandIdentifier(u.nameParts) - if (CatalogV2Util.isSessionCatalog(catalog)) { - resolveV1Function(ident.asFunctionIdentifier, u.arguments, u) - } else { - resolveV2Function(catalog.asFunctionCatalog, ident, u.arguments, u) + catalog.asFunctionCatalog.loadFunction(ident) match { + case V1Function(_) => + // this triggers the second time v1 function resolution but should be cheap + // (no RPC to external catalog), since the metadata has been already cached + // in FunctionRegistry during the above `catalog.loadFunction` call. + resolveV1Function(ident.asFunctionIdentifier, u.arguments, u) + case unboundV2Func => + resolveV2Function(unboundV2Func, u.arguments, u) } } } @@ -272,11 +275,9 @@ class FunctionResolution( } private def resolveV2Function( - catalog: FunctionCatalog, - ident: Identifier, + unbound: UnboundFunction, arguments: Seq[Expression], u: UnresolvedFunction): Expression = { - val unbound = catalog.loadFunction(ident) val inputType = StructType(arguments.zipWithIndex.map { case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable) }) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out index 94fff8f586972..95639c72a0ad4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -972,11 +972,11 @@ VALUES(IDENTIFIER('a.b.c.d')()) -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", - "sqlState" : "42601", + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`", - "limit" : "2" + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index e6a406072c48b..e3150b1996583 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -972,11 +972,11 @@ VALUES(IDENTIFIER('a.b.c.d')()) -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", - "sqlState" : "42601", + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`", - "limit" : "2" + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out index 6a99be0570100..13a4b43fd0589 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -1112,11 +1112,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", - "sqlState" : "42601", + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`", - "limit" : "2" + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 0c0473791201f..beeb3b13fe1ee 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -1112,11 +1112,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", - "sqlState" : "42601", + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`", - "limit" : "2" + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 8959b285b0283..bc6ceeb24593b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -168,6 +168,10 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio spark.sessionState.catalogManager.catalog(name) } + protected def sessionCatalog: Catalog = { + catalog(SESSION_CATALOG_NAME).asInstanceOf[Catalog] + } + protected val v2Format: String = classOf[FakeV2ProviderWithCustomSchema].getName protected val catalogClassName: String = classOf[InMemoryTableSessionCatalog].getName @@ -178,7 +182,9 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio override def afterEach(): Unit = { super.afterEach() - catalog(SESSION_CATALOG_NAME).asInstanceOf[Catalog].clearTables() + sessionCatalog.checkUsage() + sessionCatalog.clearTables() + sessionCatalog.clearFunctions() spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala index c6f2da686fe96..366528e46ff23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala @@ -702,127 +702,127 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { comparePlans(df1.queryExecution.optimizedPlan, df2.queryExecution.optimizedPlan) checkAnswer(df1, Row(3) :: Nil) } +} - private case object StrLenDefault extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_default" +case object StrLenDefault extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_default" - override def produceResult(input: InternalRow): Int = { - val s = input.getString(0) - s.length - } + override def produceResult(input: InternalRow): Int = { + val s = input.getString(0) + s.length } +} - case object StrLenMagic extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_magic" +case object StrLenMagic extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_magic" - def invoke(input: UTF8String): Int = { - input.toString.length - } + def invoke(input: UTF8String): Int = { + input.toString.length } +} - case object StrLenBadMagic extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_bad_magic" +case object StrLenBadMagic extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_bad_magic" - def invoke(input: String): Int = { - input.length - } + def invoke(input: String): Int = { + input.length } +} - case object StrLenBadMagicWithDefault extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_bad_magic" - - def invoke(input: String): Int = { - input.length - } +case object StrLenBadMagicWithDefault extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_bad_magic" - override def produceResult(input: InternalRow): Int = { - val s = input.getString(0) - s.length - } + def invoke(input: String): Int = { + input.length } - private case object StrLenNoImpl extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_noimpl" + override def produceResult(input: InternalRow): Int = { + val s = input.getString(0) + s.length } +} - // input type doesn't match arguments accepted by `UnboundFunction.bind` - private case object StrLenBadInputTypes extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType, IntegerType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_bad_input_types" - } +case object StrLenNoImpl extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_noimpl" +} - private case object BadBoundFunction extends BoundFunction { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "bad_bound_func" - } +// input type doesn't match arguments accepted by `UnboundFunction.bind` +case object StrLenBadInputTypes extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType, IntegerType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_bad_input_types" +} - object UnboundDecimalAverage extends UnboundFunction { - override def name(): String = "decimal_avg" +case object BadBoundFunction extends BoundFunction { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "bad_bound_func" +} - override def bind(inputType: StructType): BoundFunction = { - if (inputType.fields.length > 1) { - throw new UnsupportedOperationException("Too many arguments") - } +object UnboundDecimalAverage extends UnboundFunction { + override def name(): String = "decimal_avg" - // put interval type here for testing purpose - inputType.fields(0).dataType match { - case _: NumericType | _: DayTimeIntervalType => DecimalAverage - case dataType => - throw new UnsupportedOperationException(s"Unsupported input type: $dataType") - } + override def bind(inputType: StructType): BoundFunction = { + if (inputType.fields.length > 1) { + throw new UnsupportedOperationException("Too many arguments") } - override def description(): String = - "decimal_avg: produces an average using decimal division" + // put interval type here for testing purpose + inputType.fields(0).dataType match { + case _: NumericType | _: DayTimeIntervalType => DecimalAverage + case dataType => + throw new UnsupportedOperationException(s"Unsupported input type: $dataType") + } } - object DecimalAverage extends AggregateFunction[(Decimal, Int), Decimal] { - override def name(): String = "decimal_avg" - override def inputTypes(): Array[DataType] = Array(DecimalType.SYSTEM_DEFAULT) - override def resultType(): DataType = DecimalType.SYSTEM_DEFAULT + override def description(): String = + "decimal_avg: produces an average using decimal division" +} - override def newAggregationState(): (Decimal, Int) = (Decimal.ZERO, 0) +object DecimalAverage extends AggregateFunction[(Decimal, Int), Decimal] { + override def name(): String = "decimal_avg" + override def inputTypes(): Array[DataType] = Array(DecimalType.SYSTEM_DEFAULT) + override def resultType(): DataType = DecimalType.SYSTEM_DEFAULT - override def update(state: (Decimal, Int), input: InternalRow): (Decimal, Int) = { - if (input.isNullAt(0)) { - state - } else { - val l = input.getDecimal(0, DecimalType.SYSTEM_DEFAULT.precision, - DecimalType.SYSTEM_DEFAULT.scale) - state match { - case (_, d) if d == 0 => - (l, 1) - case (total, count) => - (total + l, count + 1) - } - } - } + override def newAggregationState(): (Decimal, Int) = (Decimal.ZERO, 0) - override def merge(leftState: (Decimal, Int), rightState: (Decimal, Int)): (Decimal, Int) = { - (leftState._1 + rightState._1, leftState._2 + rightState._2) + override def update(state: (Decimal, Int), input: InternalRow): (Decimal, Int) = { + if (input.isNullAt(0)) { + state + } else { + val l = input.getDecimal(0, DecimalType.SYSTEM_DEFAULT.precision, + DecimalType.SYSTEM_DEFAULT.scale) + state match { + case (_, d) if d == 0 => + (l, 1) + case (total, count) => + (total + l, count + 1) + } } + } - override def produceResult(state: (Decimal, Int)): Decimal = state._1 / Decimal(state._2) + override def merge(leftState: (Decimal, Int), rightState: (Decimal, Int)): (Decimal, Int) = { + (leftState._1 + rightState._1, leftState._2 + rightState._2) } - object NoImplAverage extends UnboundFunction { - override def name(): String = "no_impl_avg" - override def description(): String = name() + override def produceResult(state: (Decimal, Int)): Decimal = state._1 / Decimal(state._2) +} + +object NoImplAverage extends UnboundFunction { + override def name(): String = "no_impl_avg" + override def description(): String = name() - override def bind(inputType: StructType): BoundFunction = { - throw SparkUnsupportedOperationException() - } + override def bind(inputType: StructType): BoundFunction = { + throw SparkUnsupportedOperationException() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index 7463eb34d17ff..dcc49b252fdbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.connector -import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.{DataFrame, Row, SaveMode} import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable, Table, TableCatalog} class DataSourceV2SQLSessionCatalogSuite @@ -79,4 +79,11 @@ class DataSourceV2SQLSessionCatalogSuite assert(getTableMetadata("default.t").columns().map(_.name()) === Seq("c2", "c1")) } } + + test("SPARK-54760: DelegatingCatalogExtension supports both V1 and V2 functions") { + sessionCatalog.createFunction(Identifier.of(Array("ns"), "strlen"), StrLen(StrLenDefault)) + checkAnswer( + sql("SELECT char_length('Hello') as v1, ns.strlen('Spark') as v2"), + Row(5, 5)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 6b5bd982ee5a6..ef4128c297228 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -52,6 +52,10 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with spark.sessionState.catalogManager.catalog(name).asInstanceOf[TableCatalog] } + protected def sessionCatalog: InMemoryTableSessionCatalog = { + catalog(SESSION_CATALOG_NAME).asInstanceOf[InMemoryTableSessionCatalog] + } + private implicit def stringToIdentifier(value: String): Identifier = { Identifier.of(Array.empty, value) } @@ -65,7 +69,8 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with override def afterEach(): Unit = { super.afterEach() - Try(catalog(SESSION_CATALOG_NAME).asInstanceOf[InMemoryTableSessionCatalog].clearTables()) + Try(sessionCatalog.checkUsage()) + Try(sessionCatalog.clearTables()) catalog(catalogName).listTables(Array.empty).foreach( catalog(catalogName).dropTable(_)) spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) @@ -146,7 +151,7 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with val dfw = df.write.format(format).mode(SaveMode.Ignore).option("name", "t1") dfw.save() - val table = catalog(SESSION_CATALOG_NAME).loadTable(Identifier.of(Array("default"), "t1")) + val table = sessionCatalog.loadTable(Identifier.of(Array("default"), "t1")) assert(table.partitioning().isEmpty, "Partitioning should be empty") assert(table.columns() sameElements Array(Column.create("id", LongType)), "Schema did not match") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index 2254abef3fcb6..6a82dca9cafcb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -21,21 +21,32 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean import scala.jdk.CollectionConverters._ +import scala.util.{Failure, Success, Try} +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, DelegatingCatalogExtension, Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** * A V2SessionCatalog implementation that can be extended to generate arbitrary `Table` definitions * for testing DDL as well as write operations (through df.write.saveAsTable, df.write.insertInto - * and SQL). + * and SQL), also supports v2 function operations. */ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { protected val tables: java.util.Map[Identifier, T] = new ConcurrentHashMap[Identifier, T]() + protected val functions: java.util.Map[Identifier, UnboundFunction] = + new ConcurrentHashMap[Identifier, UnboundFunction]() private val tableCreated: AtomicBoolean = new AtomicBoolean(false) + private val funcCreated: AtomicBoolean = new AtomicBoolean(false) + + def checkUsage(): Unit = { + assert(tableCreated.get || funcCreated.get, + "Either tables or functions are not created, maybe didn't use the session catalog code path?") + } private def addTable(ident: Identifier, table: T): Unit = { tableCreated.set(true) @@ -96,13 +107,54 @@ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends Delegating } def clearTables(): Unit = { - assert( - tableCreated.get, - "Tables are not created, maybe didn't use the session catalog code path?") tables.keySet().asScala.foreach(super.dropTable) tables.clear() tableCreated.set(false) } + + override def listFunctions(namespace: Array[String]): Array[Identifier] = { + (Try(listFunctions0(namespace)), Try(super.listFunctions(namespace))) match { + case (Success(v2), Success(v1)) => v2 ++ v1 + case (Success(v2), Failure(_)) => v2 + case (Failure(_), Success(v1)) => v1 + case (Failure(_), Failure(_)) => + throw new NoSuchNamespaceException(namespace) + } + } + + private def listFunctions0(namespace: Array[String]): Array[Identifier] = { + if (namespace.isEmpty || namespaceExists(namespace)) { + functions.keySet.asScala.filter(_.namespace.sameElements(namespace)).toArray + } else { + throw new NoSuchNamespaceException(namespace) + } + } + + override def loadFunction(ident: Identifier): UnboundFunction = { + Option(functions.get(ident)) match { + case Some(func) => func + case _ => + super.loadFunction(ident) + } + } + + override def functionExists(ident: Identifier): Boolean = { + functions.containsKey(ident) || super.functionExists(ident) + } + + def createFunction(ident: Identifier, fn: UnboundFunction): UnboundFunction = { + funcCreated.set(true) + functions.put(ident, fn) + } + + def dropFunction(ident: Identifier): Unit = { + functions.remove(ident) + } + + def clearFunctions(): Unit = { + functions.clear() + funcCreated.set(false) + } } object TestV2SessionCatalogBase { From eec9f8fd4c7c08330b19f971df891969b9604697 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 25 Dec 2025 13:23:27 +0800 Subject: [PATCH 324/400] [SPARK-54820][PYTHON][4.0][4.1] Make pandas_on_spark_type compatible with numpy 2.4.0 backport https://github.com/apache/spark/pull/53580 to 4.0 and 4.1, to restore https://github.com/apache/spark/actions/runs/20493084511/job/58888601618 Closes #53604 from zhengruifeng/bp_np_24. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/pandas/typedef/typehints.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/pandas/typedef/typehints.py b/python/pyspark/pandas/typedef/typehints.py index 48545d124b2d8..a4ed9f996fe47 100644 --- a/python/pyspark/pandas/typedef/typehints.py +++ b/python/pyspark/pandas/typedef/typehints.py @@ -342,7 +342,7 @@ def pandas_on_spark_type(tpe: Union[str, type, Dtype]) -> Tuple[Dtype, types.Dat try: dtype = pandas_dtype(tpe) spark_type = as_spark_type(dtype) - except TypeError: + except (TypeError, ValueError): spark_type = as_spark_type(tpe) dtype = spark_type_to_pandas_dtype(spark_type) return dtype, spark_type From 6d7ff7670cbabca8b57984584b751f479a882bd4 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Fri, 26 Dec 2025 22:06:13 +0800 Subject: [PATCH 325/400] [SPARK-54843][SQL] Try_to_number expression not working for empty string input ### What changes were proposed in this pull request? Catching the case in `ToNumberParser` when the input string only consists of whitespace, preventing a failure with an internal error later on when trying to create `BigDecimal`. ### Why are the changes needed? Without this change passing an empty string (`select try_to_number('', '99')`) would fail with the following exception: ``` JVM stacktrace: java.lang.NumberFormatException at java.base/java.math.BigDecimal.(BigDecimal.java:692) at java.base/java.math.BigDecimal.(BigDecimal.java:471) at java.base/java.math.BigDecimal.(BigDecimal.java:900) at org.apache.spark.sql.catalyst.util.ToNumberParser.parseResultToDecimalValue(ToNumberParser.scala:627) at org.apache.spark.sql.catalyst.util.ToNumberParser.parse(ToNumberParser.scala:499) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53609 from stefankandic/numFormatFix. Authored-by: Stefan Kandic Signed-off-by: Wenchen Fan (cherry picked from commit df720c17958482796740089595366d7201bc2557) Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/ToNumberParser.scala | 3 +++ .../spark/sql/StringFunctionsSuite.scala | 20 +++++++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala index ea2f48fafc0dd..ffcf8ba2cb93d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala @@ -495,6 +495,9 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali // If we have consumed all the tokens in the format string, but characters remain unconsumed // in the input string, then the input string does not match the format string. formatMatchFailure(input, numberFormat) + } else if (parsedBeforeDecimalPoint.isEmpty && parsedAfterDecimalPoint.isEmpty) { + // If no digits were collected (e.g. input was all whitespace), treat as format match failure. + formatMatchFailure(input, numberFormat) } else { parseResultToDecimalValue(negateResult) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index a1de322ac298c..ff0ee19ae971a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -1342,6 +1342,26 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df.select(try_to_number($"value", lit("$99.99"))), Seq(Row(null))) } + test("try_to_number with whitespace-only input should return NULL") { + // Empty string + checkAnswer(sql("select try_to_number('', '99')"), Seq(Row(null))) + checkAnswer(sql("select try_to_number('', '999')"), Seq(Row(null))) + + // Spaces only + checkAnswer(sql("select try_to_number(' ', '99')"), Seq(Row(null))) + checkAnswer(sql("select try_to_number(' ', '9')"), Seq(Row(null))) + + // Different whitespace characters (tabs, newlines) + checkAnswer(sql("select try_to_number('\t\t', '99')"), Seq(Row(null))) + checkAnswer(sql("select try_to_number('\n\n', '99')"), Seq(Row(null))) + checkAnswer(sql("select try_to_number(' \t\n ', '99')"), Seq(Row(null))) + + // With format strings containing decimal points, dollar signs, etc. + checkAnswer(sql("select try_to_number(' ', '$99.99')"), Seq(Row(null))) + checkAnswer(sql("select try_to_number('', '999.99')"), Seq(Row(null))) + checkAnswer(sql("select try_to_number('\t', '9,999')"), Seq(Row(null))) + } + test("SPARK-44905: stateful lastRegex causes NullPointerException on eval for regexp_replace") { val df = sql("select regexp_replace('', '[a\\\\d]{0, 2}', 'x')") intercept[SparkRuntimeException](df.queryExecution.optimizedPlan) From fb13ecd279440cc08fca664ddcd224b8927d796e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 27 Dec 2025 09:22:05 +0800 Subject: [PATCH 326/400] [SPARK-54847][BUILD] unify the proto/antlr output folder between sbt and maven ### What changes were proposed in this pull request? People may use sbt and maven together: local test with sbt, IDE with Maven. As of today this will trigger duplicated class issue during compilation, because sbt and Maven use different output folder for generated proto/antlr java classes. This PR updates the sbt build to use the same folder as Maven for proto/antlr output. ### Why are the changes needed? Improve dev experience. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test ### Was this patch authored or co-authored using generative AI tooling? cursor 2.2.43 Closes #53616 from cloud-fan/shim. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 174b62d01212c8032c9ef63cd2beddcc80d0d8d6) Signed-off-by: Wenchen Fan --- .sbtopts | 20 ++++++++++++++++++++ project/SparkBuild.scala | 21 ++++++++++++++------- 2 files changed, 34 insertions(+), 7 deletions(-) create mode 100644 .sbtopts diff --git a/.sbtopts b/.sbtopts new file mode 100644 index 0000000000000..3516fc4bd7ebc --- /dev/null +++ b/.sbtopts @@ -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. +# + +-J-Xmx8g +-J-Xms8g +-J-XX:MaxMetaspaceSize=1g diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index fec6a3346f7d2..988d5c2e64393 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -646,8 +646,10 @@ object Core { "com.google.protobuf" % "protobuf-java" % protoVersion % "protobuf" ) }, + // Use Maven's output directory so sbt and Maven can share generated sources. + // Core uses protoc-jar-maven-plugin which outputs to target/generated-sources. (Compile / PB.targets) := Seq( - PB.gens.java -> (Compile / sourceManaged).value + PB.gens.java -> target.value / "generated-sources" ) ) ++ { val sparkProtocExecPath = sys.props.get("spark.protoc.executable.path") @@ -732,19 +734,20 @@ object SparkConnectCommon { ) ++ { val sparkProtocExecPath = sys.props.get("spark.protoc.executable.path") val connectPluginExecPath = sys.props.get("connect.plugin.executable.path") + // Use Maven's output directory so sbt and Maven can share generated sources if (sparkProtocExecPath.isDefined && connectPluginExecPath.isDefined) { Seq( (Compile / PB.targets) := Seq( - PB.gens.java -> (Compile / sourceManaged).value, - PB.gens.plugin(name = "grpc-java", path = connectPluginExecPath.get) -> (Compile / sourceManaged).value + PB.gens.java -> target.value / "generated-sources" / "protobuf" / "java", + PB.gens.plugin(name = "grpc-java", path = connectPluginExecPath.get) -> target.value / "generated-sources" / "protobuf" / "grpc-java" ), PB.protocExecutable := file(sparkProtocExecPath.get) ) } else { Seq( (Compile / PB.targets) := Seq( - PB.gens.java -> (Compile / sourceManaged).value, - PB.gens.plugin("grpc-java") -> (Compile / sourceManaged).value + PB.gens.java -> target.value / "generated-sources" / "protobuf" / "java", + PB.gens.plugin("grpc-java") -> target.value / "generated-sources" / "protobuf" / "grpc-java" ) ) } @@ -1283,7 +1286,9 @@ object SqlApi { (Antlr4 / antlr4PackageName) := Some("org.apache.spark.sql.catalyst.parser"), (Antlr4 / antlr4GenListener) := true, (Antlr4 / antlr4GenVisitor) := true, - (Antlr4 / antlr4TreatWarningsAsErrors) := true + (Antlr4 / antlr4TreatWarningsAsErrors) := true, + // Use Maven's output directory so sbt and Maven can share generated sources + (Antlr4 / javaSource) := target.value / "generated-sources" / "antlr4" ) } @@ -1300,8 +1305,10 @@ object SQL { "com.google.protobuf" % "protobuf-java" % protoVersion % "protobuf" ) }, + // Use Maven's output directory so sbt and Maven can share generated sources. + // sql/core uses protoc-jar-maven-plugin which outputs to target/generated-sources. (Compile / PB.targets) := Seq( - PB.gens.java -> (Compile / sourceManaged).value + PB.gens.java -> target.value / "generated-sources" ) ) ++ { val sparkProtocExecPath = sys.props.get("spark.protoc.executable.path") From cb76c664d6d277631973cf152787339a1d89bbd2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 27 Dec 2025 09:24:18 +0800 Subject: [PATCH 327/400] [SPARK-54851][BUILD] Support generating bloop files via sbt ### What changes were proposed in this pull request? Some IDEs such as cursor/VSCode works better with bloop as the build server. This PR supports generating the bloop config files via sbt. ### Why are the changes needed? better IDE support ### Does this PR introduce _any_ user-facing change? no, dev only ### How was this patch tested? manual test ### Was this patch authored or co-authored using generative AI tooling? cursor 2.2.43 Closes #53621 from cloud-fan/bloop. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit eec092c9f9d1ad3df0390b1ea56340c1103ba887) Signed-off-by: Wenchen Fan --- project/SparkBuild.scala | 24 ++++++++++++++++++++++++ project/plugins.sbt | 2 ++ 2 files changed, 26 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 988d5c2e64393..6c37944fb48aa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1225,8 +1225,15 @@ object ExcludedDependencies { * client dependencies. */ object ExcludeShims { + import bloop.integrations.sbt.BloopKeys + val shimmedProjects = Set("spark-sql-api", "spark-connect-common", "spark-connect-client-jdbc", "spark-connect-client-jvm") val classPathFilter = TaskKey[Classpath => Classpath]("filter for classpath") + + // Filter for bloopInternalClasspath which is Seq[(File, File)] + type BloopClasspath = Seq[(java.io.File, java.io.File)] + val bloopClasspathFilter = TaskKey[BloopClasspath => BloopClasspath]("filter for bloop classpath") + lazy val settings = Seq( classPathFilter := { if (!shimmedProjects(moduleName.value)) { @@ -1235,6 +1242,16 @@ object ExcludeShims { identity _ } }, + bloopClasspathFilter := { + if (!shimmedProjects(moduleName.value)) { + // Note: bloop output directories use "connect-shims" (without "spark-" prefix) + cp => cp.filterNot { case (f1, f2) => + f1.getPath.contains("connect-shims") || f2.getPath.contains("connect-shims") + } + } else { + identity _ + } + }, Compile / internalDependencyClasspath := classPathFilter.value((Compile / internalDependencyClasspath).value), Compile / internalDependencyAsJars := @@ -1247,6 +1264,13 @@ object ExcludeShims { classPathFilter.value((Test / internalDependencyClasspath).value), Test / internalDependencyAsJars := classPathFilter.value((Test / internalDependencyAsJars).value), + // Filter bloop's internal classpath for correct IDE integration + Compile / BloopKeys.bloopInternalClasspath := + bloopClasspathFilter.value((Compile / BloopKeys.bloopInternalClasspath).value), + Runtime / BloopKeys.bloopInternalClasspath := + bloopClasspathFilter.value((Runtime / BloopKeys.bloopInternalClasspath).value), + Test / BloopKeys.bloopInternalClasspath := + bloopClasspathFilter.value((Test / BloopKeys.bloopInternalClasspath).value), ) } diff --git a/project/plugins.sbt b/project/plugins.sbt index fe18d16c48227..0b7e04222e306 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -46,3 +46,5 @@ addSbtPlugin("com.github.sbt.junit" % "sbt-jupiter-interface" % "0.17.0") addSbtPlugin("com.thesamet" % "sbt-protoc" % "1.0.7") addSbtPlugin("com.here.platform" % "sbt-bom" % "1.0.29") + +addSbtPlugin("ch.epfl.scala" % "sbt-bloop" % "2.0.17") From 482b03cbd1269d53ccf5ff46430d920f66d38df2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 27 Dec 2025 13:55:36 +0800 Subject: [PATCH 328/400] Revert "[SPARK-54760][SQL] DelegatingCatalogExtension as session catalog supports both V1 and V2 functions" This reverts commit 51042d66d8e88bdd1ee1a150b775d681e45d69d4. --- .../analysis/FunctionResolution.scala | 23 ++- .../identifier-clause-legacy.sql.out | 8 +- .../identifier-clause.sql.out | 8 +- .../results/identifier-clause-legacy.sql.out | 8 +- .../results/identifier-clause.sql.out | 8 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 8 +- .../connector/DataSourceV2FunctionSuite.scala | 182 +++++++++--------- .../DataSourceV2SQLSessionCatalogSuite.scala | 9 +- .../SupportsCatalogOptionsSuite.scala | 9 +- .../connector/TestV2SessionCatalogBase.scala | 60 +----- 10 files changed, 126 insertions(+), 197 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala index 8d6e2931a73b9..800126e0030e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala @@ -26,16 +26,17 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{ CatalogManager, + CatalogV2Util, + FunctionCatalog, + Identifier, LookupCatalog } import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.functions.{ AggregateFunction => V2AggregateFunction, - ScalarFunction, - UnboundFunction + ScalarFunction } import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} -import org.apache.spark.sql.internal.connector.V1Function import org.apache.spark.sql.types._ class FunctionResolution( @@ -51,14 +52,10 @@ class FunctionResolution( resolveBuiltinOrTempFunction(u.nameParts, u.arguments, u).getOrElse { val CatalogAndIdentifier(catalog, ident) = relationResolution.expandIdentifier(u.nameParts) - catalog.asFunctionCatalog.loadFunction(ident) match { - case V1Function(_) => - // this triggers the second time v1 function resolution but should be cheap - // (no RPC to external catalog), since the metadata has been already cached - // in FunctionRegistry during the above `catalog.loadFunction` call. - resolveV1Function(ident.asFunctionIdentifier, u.arguments, u) - case unboundV2Func => - resolveV2Function(unboundV2Func, u.arguments, u) + if (CatalogV2Util.isSessionCatalog(catalog)) { + resolveV1Function(ident.asFunctionIdentifier, u.arguments, u) + } else { + resolveV2Function(catalog.asFunctionCatalog, ident, u.arguments, u) } } } @@ -275,9 +272,11 @@ class FunctionResolution( } private def resolveV2Function( - unbound: UnboundFunction, + catalog: FunctionCatalog, + ident: Identifier, arguments: Seq[Expression], u: UnresolvedFunction): Expression = { + val unbound = catalog.loadFunction(ident) val inputType = StructType(arguments.zipWithIndex.map { case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable) }) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out index 95639c72a0ad4..94fff8f586972 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -972,11 +972,11 @@ VALUES(IDENTIFIER('a.b.c.d')()) -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", - "sqlState" : "42K05", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", "messageParameters" : { - "namespace" : "`a`.`b`.`c`", - "sessionCatalog" : "spark_catalog" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index e3150b1996583..e6a406072c48b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -972,11 +972,11 @@ VALUES(IDENTIFIER('a.b.c.d')()) -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", - "sqlState" : "42K05", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", "messageParameters" : { - "namespace" : "`a`.`b`.`c`", - "sessionCatalog" : "spark_catalog" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out index 13a4b43fd0589..6a99be0570100 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -1112,11 +1112,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", - "sqlState" : "42K05", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", "messageParameters" : { - "namespace" : "`a`.`b`.`c`", - "sessionCatalog" : "spark_catalog" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index beeb3b13fe1ee..0c0473791201f 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -1112,11 +1112,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", - "sqlState" : "42K05", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", "messageParameters" : { - "namespace" : "`a`.`b`.`c`", - "sessionCatalog" : "spark_catalog" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index bc6ceeb24593b..8959b285b0283 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -168,10 +168,6 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio spark.sessionState.catalogManager.catalog(name) } - protected def sessionCatalog: Catalog = { - catalog(SESSION_CATALOG_NAME).asInstanceOf[Catalog] - } - protected val v2Format: String = classOf[FakeV2ProviderWithCustomSchema].getName protected val catalogClassName: String = classOf[InMemoryTableSessionCatalog].getName @@ -182,9 +178,7 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio override def afterEach(): Unit = { super.afterEach() - sessionCatalog.checkUsage() - sessionCatalog.clearTables() - sessionCatalog.clearFunctions() + catalog(SESSION_CATALOG_NAME).asInstanceOf[Catalog].clearTables() spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala index 366528e46ff23..c6f2da686fe96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala @@ -702,127 +702,127 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { comparePlans(df1.queryExecution.optimizedPlan, df2.queryExecution.optimizedPlan) checkAnswer(df1, Row(3) :: Nil) } -} -case object StrLenDefault extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_default" + private case object StrLenDefault extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_default" - override def produceResult(input: InternalRow): Int = { - val s = input.getString(0) - s.length + override def produceResult(input: InternalRow): Int = { + val s = input.getString(0) + s.length + } } -} -case object StrLenMagic extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_magic" + case object StrLenMagic extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_magic" - def invoke(input: UTF8String): Int = { - input.toString.length + def invoke(input: UTF8String): Int = { + input.toString.length + } } -} -case object StrLenBadMagic extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_bad_magic" + case object StrLenBadMagic extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_bad_magic" - def invoke(input: String): Int = { - input.length + def invoke(input: String): Int = { + input.length + } } -} -case object StrLenBadMagicWithDefault extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_bad_magic" + case object StrLenBadMagicWithDefault extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_bad_magic" + + def invoke(input: String): Int = { + input.length + } - def invoke(input: String): Int = { - input.length + override def produceResult(input: InternalRow): Int = { + val s = input.getString(0) + s.length + } } - override def produceResult(input: InternalRow): Int = { - val s = input.getString(0) - s.length + private case object StrLenNoImpl extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_noimpl" } -} -case object StrLenNoImpl extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_noimpl" -} + // input type doesn't match arguments accepted by `UnboundFunction.bind` + private case object StrLenBadInputTypes extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType, IntegerType) + override def resultType(): DataType = IntegerType + override def name(): String = "strlen_bad_input_types" + } -// input type doesn't match arguments accepted by `UnboundFunction.bind` -case object StrLenBadInputTypes extends ScalarFunction[Int] { - override def inputTypes(): Array[DataType] = Array(StringType, IntegerType) - override def resultType(): DataType = IntegerType - override def name(): String = "strlen_bad_input_types" -} + private case object BadBoundFunction extends BoundFunction { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "bad_bound_func" + } -case object BadBoundFunction extends BoundFunction { - override def inputTypes(): Array[DataType] = Array(StringType) - override def resultType(): DataType = IntegerType - override def name(): String = "bad_bound_func" -} + object UnboundDecimalAverage extends UnboundFunction { + override def name(): String = "decimal_avg" -object UnboundDecimalAverage extends UnboundFunction { - override def name(): String = "decimal_avg" + override def bind(inputType: StructType): BoundFunction = { + if (inputType.fields.length > 1) { + throw new UnsupportedOperationException("Too many arguments") + } - override def bind(inputType: StructType): BoundFunction = { - if (inputType.fields.length > 1) { - throw new UnsupportedOperationException("Too many arguments") + // put interval type here for testing purpose + inputType.fields(0).dataType match { + case _: NumericType | _: DayTimeIntervalType => DecimalAverage + case dataType => + throw new UnsupportedOperationException(s"Unsupported input type: $dataType") + } } - // put interval type here for testing purpose - inputType.fields(0).dataType match { - case _: NumericType | _: DayTimeIntervalType => DecimalAverage - case dataType => - throw new UnsupportedOperationException(s"Unsupported input type: $dataType") - } + override def description(): String = + "decimal_avg: produces an average using decimal division" } - override def description(): String = - "decimal_avg: produces an average using decimal division" -} - -object DecimalAverage extends AggregateFunction[(Decimal, Int), Decimal] { - override def name(): String = "decimal_avg" - override def inputTypes(): Array[DataType] = Array(DecimalType.SYSTEM_DEFAULT) - override def resultType(): DataType = DecimalType.SYSTEM_DEFAULT + object DecimalAverage extends AggregateFunction[(Decimal, Int), Decimal] { + override def name(): String = "decimal_avg" + override def inputTypes(): Array[DataType] = Array(DecimalType.SYSTEM_DEFAULT) + override def resultType(): DataType = DecimalType.SYSTEM_DEFAULT - override def newAggregationState(): (Decimal, Int) = (Decimal.ZERO, 0) + override def newAggregationState(): (Decimal, Int) = (Decimal.ZERO, 0) - override def update(state: (Decimal, Int), input: InternalRow): (Decimal, Int) = { - if (input.isNullAt(0)) { - state - } else { - val l = input.getDecimal(0, DecimalType.SYSTEM_DEFAULT.precision, - DecimalType.SYSTEM_DEFAULT.scale) - state match { - case (_, d) if d == 0 => - (l, 1) - case (total, count) => - (total + l, count + 1) + override def update(state: (Decimal, Int), input: InternalRow): (Decimal, Int) = { + if (input.isNullAt(0)) { + state + } else { + val l = input.getDecimal(0, DecimalType.SYSTEM_DEFAULT.precision, + DecimalType.SYSTEM_DEFAULT.scale) + state match { + case (_, d) if d == 0 => + (l, 1) + case (total, count) => + (total + l, count + 1) + } } } - } - override def merge(leftState: (Decimal, Int), rightState: (Decimal, Int)): (Decimal, Int) = { - (leftState._1 + rightState._1, leftState._2 + rightState._2) - } + override def merge(leftState: (Decimal, Int), rightState: (Decimal, Int)): (Decimal, Int) = { + (leftState._1 + rightState._1, leftState._2 + rightState._2) + } - override def produceResult(state: (Decimal, Int)): Decimal = state._1 / Decimal(state._2) -} + override def produceResult(state: (Decimal, Int)): Decimal = state._1 / Decimal(state._2) + } -object NoImplAverage extends UnboundFunction { - override def name(): String = "no_impl_avg" - override def description(): String = name() + object NoImplAverage extends UnboundFunction { + override def name(): String = "no_impl_avg" + override def description(): String = name() - override def bind(inputType: StructType): BoundFunction = { - throw SparkUnsupportedOperationException() + override def bind(inputType: StructType): BoundFunction = { + throw SparkUnsupportedOperationException() + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index dcc49b252fdbd..7463eb34d17ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.connector -import org.apache.spark.sql.{DataFrame, Row, SaveMode} +import org.apache.spark.sql.{DataFrame, SaveMode} import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable, Table, TableCatalog} class DataSourceV2SQLSessionCatalogSuite @@ -79,11 +79,4 @@ class DataSourceV2SQLSessionCatalogSuite assert(getTableMetadata("default.t").columns().map(_.name()) === Seq("c2", "c1")) } } - - test("SPARK-54760: DelegatingCatalogExtension supports both V1 and V2 functions") { - sessionCatalog.createFunction(Identifier.of(Array("ns"), "strlen"), StrLen(StrLenDefault)) - checkAnswer( - sql("SELECT char_length('Hello') as v1, ns.strlen('Spark') as v2"), - Row(5, 5)) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index ef4128c297228..6b5bd982ee5a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -52,10 +52,6 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with spark.sessionState.catalogManager.catalog(name).asInstanceOf[TableCatalog] } - protected def sessionCatalog: InMemoryTableSessionCatalog = { - catalog(SESSION_CATALOG_NAME).asInstanceOf[InMemoryTableSessionCatalog] - } - private implicit def stringToIdentifier(value: String): Identifier = { Identifier.of(Array.empty, value) } @@ -69,8 +65,7 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with override def afterEach(): Unit = { super.afterEach() - Try(sessionCatalog.checkUsage()) - Try(sessionCatalog.clearTables()) + Try(catalog(SESSION_CATALOG_NAME).asInstanceOf[InMemoryTableSessionCatalog].clearTables()) catalog(catalogName).listTables(Array.empty).foreach( catalog(catalogName).dropTable(_)) spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) @@ -151,7 +146,7 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with val dfw = df.write.format(format).mode(SaveMode.Ignore).option("name", "t1") dfw.save() - val table = sessionCatalog.loadTable(Identifier.of(Array("default"), "t1")) + val table = catalog(SESSION_CATALOG_NAME).loadTable(Identifier.of(Array("default"), "t1")) assert(table.partitioning().isEmpty, "Partitioning should be empty") assert(table.columns() sameElements Array(Column.create("id", LongType)), "Schema did not match") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index 6a82dca9cafcb..2254abef3fcb6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -21,32 +21,21 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean import scala.jdk.CollectionConverters._ -import scala.util.{Failure, Success, Try} -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, DelegatingCatalogExtension, Identifier, Table, TableCatalog} -import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** * A V2SessionCatalog implementation that can be extended to generate arbitrary `Table` definitions * for testing DDL as well as write operations (through df.write.saveAsTable, df.write.insertInto - * and SQL), also supports v2 function operations. + * and SQL). */ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { protected val tables: java.util.Map[Identifier, T] = new ConcurrentHashMap[Identifier, T]() - protected val functions: java.util.Map[Identifier, UnboundFunction] = - new ConcurrentHashMap[Identifier, UnboundFunction]() private val tableCreated: AtomicBoolean = new AtomicBoolean(false) - private val funcCreated: AtomicBoolean = new AtomicBoolean(false) - - def checkUsage(): Unit = { - assert(tableCreated.get || funcCreated.get, - "Either tables or functions are not created, maybe didn't use the session catalog code path?") - } private def addTable(ident: Identifier, table: T): Unit = { tableCreated.set(true) @@ -107,54 +96,13 @@ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends Delegating } def clearTables(): Unit = { + assert( + tableCreated.get, + "Tables are not created, maybe didn't use the session catalog code path?") tables.keySet().asScala.foreach(super.dropTable) tables.clear() tableCreated.set(false) } - - override def listFunctions(namespace: Array[String]): Array[Identifier] = { - (Try(listFunctions0(namespace)), Try(super.listFunctions(namespace))) match { - case (Success(v2), Success(v1)) => v2 ++ v1 - case (Success(v2), Failure(_)) => v2 - case (Failure(_), Success(v1)) => v1 - case (Failure(_), Failure(_)) => - throw new NoSuchNamespaceException(namespace) - } - } - - private def listFunctions0(namespace: Array[String]): Array[Identifier] = { - if (namespace.isEmpty || namespaceExists(namespace)) { - functions.keySet.asScala.filter(_.namespace.sameElements(namespace)).toArray - } else { - throw new NoSuchNamespaceException(namespace) - } - } - - override def loadFunction(ident: Identifier): UnboundFunction = { - Option(functions.get(ident)) match { - case Some(func) => func - case _ => - super.loadFunction(ident) - } - } - - override def functionExists(ident: Identifier): Boolean = { - functions.containsKey(ident) || super.functionExists(ident) - } - - def createFunction(ident: Identifier, fn: UnboundFunction): UnboundFunction = { - funcCreated.set(true) - functions.put(ident, fn) - } - - def dropFunction(ident: Identifier): Unit = { - functions.remove(ident) - } - - def clearFunctions(): Unit = { - functions.clear() - funcCreated.set(false) - } } object TestV2SessionCatalogBase { From 0a0b47e2c66a59eaaeb26014d7a09c528ccc701b Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 29 Dec 2025 11:54:37 +0800 Subject: [PATCH 329/400] [SPARK-54837][DOCS] Document all built-in JDBC datasource providers ### What changes were proposed in this pull request? Update the docs of the JDBC datasource to mention all built-in providers. ### Why are the changes needed? Keep the docs up to date. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53598 from pan3793/SPARK-54837. Authored-by: Cheng Pan Signed-off-by: yangjie01 (cherry picked from commit 578ee01678e3a17a3052613fa3b57f95a29e9ffa) Signed-off-by: yangjie01 --- docs/sql-data-sources-jdbc.md | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index bf53ffa65d618..d927973e96ac3 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -389,12 +389,17 @@ Before using keytab and principal configuration option * The included JDBC driver version supports kerberos authentication with keytab. * There is a built-in connection provider which supports the used database. -There is a built-in connection providers for the following databases: +There are built-in connection providers for the following databases: +* Databricks * DB2 -* MariaDB -* MS Sql +* Derby +* H2 +* MariaDB and MySQL +* Microsoft SQL Server * Oracle * PostgreSQL +* Snowflake +* Teradata If the requirements are not met, please consider using the JdbcConnectionProvider developer API to handle custom authentication. From 1cc2478a1f37d4b76f4c4d7e215e56a2f921c8d1 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Mon, 29 Dec 2025 21:11:24 +0800 Subject: [PATCH 330/400] [SPARK-54850][SQL] Improve `extractShuffleIds` to find `AdaptiveSparkPlanExec` anywhere in plan tree ### What changes were proposed in this pull request? This PR uses `collectFirst` to find the first `AdaptiveSparkPlanExec` node anywhere in the plan tree, instead of assuming the root plan is an `AdaptiveSparkPlanExec`. ### Why are the changes needed? https://github.com/apache/spark/pull/52157 introduced the `extractShuffleIds` method in `SQLExecution` to find shuffle IDs of `SparkPlan`. Previously, the method implicitly assumed that if AQE is enabled, the `AdaptiveSparkPlanExec` would be at the root of the input. Since Spark only inserts `AdaptiveSparkPlanExec` under Command, this assumption was fine. However, the `AdaptiveSparkPlanExec` may not be the root node in Gluten. Gluten needs to insert a special physical plan to do column to row transition. By using `collectFirst`, we can correctly locate the `AdaptiveSparkPlanExec` regardless of its position in the plan tree, which improves compatibility. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass GHA. ### Was this patch authored or co-authored using generative AI tooling? No Closes #53620 from baibaichen/feature/extractShuffleIds. Authored-by: Chang chen Signed-off-by: Kent Yao (cherry picked from commit 082de7fcdc4030bd96ec5f6cd534030e961418bf) Signed-off-by: Kent Yao --- .../org/apache/spark/sql/execution/SQLExecution.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 1cab0f8d35af5..19bafeb196122 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -71,11 +71,12 @@ object SQLExecution extends Logging { } private def extractShuffleIds(plan: SparkPlan): Seq[Int] = { - plan match { + val shuffleIdsOption = plan.collectFirst { case ae: AdaptiveSparkPlanExec => ae.context.shuffleIds.asScala.keys.toSeq - case nonAdaptivePlan => - nonAdaptivePlan.collect { + } + shuffleIdsOption.getOrElse { + plan.collect { case exec: ShuffleExchangeLike => exec.shuffleId } } From 554af03a281d211a251da701bfbd53577697f117 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 30 Dec 2025 00:31:12 +0000 Subject: [PATCH 331/400] Removing test jars and class files --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 3827 -> 0 bytes core/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes data/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes data/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes dev/test-classes.txt | 7 ------- dev/test-jars.txt | 17 ----------------- .../src/test/resources/TestHelloV2_2.13.jar | Bin 4118 -> 0 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 5674 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 384581 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../artifact-tests/smallClassFileDup.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 787 -> 0 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 6119 -> 0 bytes .../test/resources/artifact-tests/Hello.class | Bin 5671 -> 0 bytes .../artifact-tests/HelloWithPackage.class | Bin 635 -> 0 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 1333 -> 0 bytes .../artifact-tests/smallClassFile.class | Bin 424 -> 0 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 5545 -> 0 bytes .../src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 35660 -> 0 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 7439 -> 0 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../test/resources/data/files/TestSerDe.jar | Bin 3838 -> 0 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 19579 -> 0 bytes 26 files changed, 24 deletions(-) delete mode 100644 core/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 core/src/test/resources/TestHelloV3_2.13.jar delete mode 100644 core/src/test/resources/TestUDTF.jar delete mode 100755 data/artifact-tests/junitLargeJar.jar delete mode 100755 data/artifact-tests/smallJar.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar delete mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar delete mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class delete mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar delete mode 100644 sql/core/src/test/resources/SPARK-33084.jar delete mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class delete mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class delete mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class delete mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class delete mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar delete mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar delete mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar delete mode 100644 sql/hive/src/test/resources/TestUDTF.jar delete mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar delete mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar deleted file mode 100644 index 0c292e7d81ad741dbc11177224b9c30519b7a362..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar deleted file mode 100644 index 514f2d5d26fd358ad5647e0e75edb8ce77b69e30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF literal 0 HcmV?d00001 diff --git a/dev/test-classes.txt b/dev/test-classes.txt index e69de29bb2d1d..2dc6b290ad4f9 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -0,0 +1,7 @@ +sql/core/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/IntSumUdf.class +sql/core/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/Hello.class +sql/core/src/test/resources/artifact-tests/HelloWithPackage.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class +sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index e69de29bb2d1d..bd8fc93bc9f0f 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -0,0 +1,17 @@ +core/src/test/resources/TestHelloV2_2.13.jar +core/src/test/resources/TestHelloV3_2.13.jar +core/src/test/resources/TestUDTF.jar +data/artifact-tests/junitLargeJar.jar +data/artifact-tests/smallJar.jar +sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar +sql/connect/client/jvm/src/test/resources/udf2.13.jar +sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar +sql/connect/common/src/test/resources/artifact-tests/smallJar.jar +sql/core/src/test/resources/SPARK-33084.jar +sql/core/src/test/resources/artifact-tests/udf_noA.jar +sql/hive-thriftserver/src/test/resources/TestUDTF.jar +sql/hive/src/test/noclasspath/hive-test-udfs.jar +sql/hive/src/test/resources/SPARK-21101-1.0.jar +sql/hive/src/test/resources/TestUDTF.jar +sql/hive/src/test/resources/data/files/TestSerDe.jar +sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..c89830f127c0c18690cec93ec229c6d3b50f7e95 GIT binary patch literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class new file mode 100644 index 0000000000000000000000000000000000000000..56725764de20539b9c16799d2bff4ee3a4e7de97 GIT binary patch literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy literal 0 HcmV?d00001 diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar new file mode 100755 index 0000000000000000000000000000000000000000..6da55d8b8520dcc03c250a605151cc0d23a45518 GIT binary patch literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar deleted file mode 100644 index 0d10f7ff03b35a1829e4f64ddbc82a0124b72f1b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19579 zcmb5V1CS`)(k0rqZQHhO+kM)$?bEhx+dOUCwr%5`@4xS#d*@BeiD8b6_vGj z=FZHuQb8IR1O)&90s;WaiCz&mfeX1F002M<5CGusQvg{}6+v1_IWc-!K{-h=QDqf6 zS+VF?L8yKP1ksm(O+&i4f>WUmE6{SG=D2-666L5CNy&wXR(4dCGN>A`&=>xjlm1gE z?wOcVE*n{x2XnEp77HuuS@*qx#N_oA3}H+GiY(s3$G9^<{$BM7A}HRdo5Vn~umq&- zg2m(%`ALckp*=`Kix8NwjdA@D;nmgcGH^E8uNDa}UFsHPbT+bKSZb+dxDfH7d2N_> zj9G#GL&{saRoanc|0!l; z=xSn1|GyU|_+N$X9L?wq?G260P3WBL4IQoie@0XNAB=W3adM_*qGM$KulM|G@*w|C zUU>1-l`bd%z&11hz~2)<*3iP1gwDv?(8(!T%|abn9mAK7OomBhu&ixSDoF!4kkYbR zyBSJru5OVaWqlcL4@oRBorAf4eTmL>=fy{#FEs}_<|qq~|AoJ~bS4|5906!LtbL=C z=SBBs=jZzk-GA#bd-2 zdns^|WB%5AUaBTvP@wWwp(mE|Z1fJNK4D2F=MZ#SoqwfzIkbGj&|TyvcL8xRkF z@=#{t5e||@x(OC~5&tn2vB@AYhHH?h4H|nO%LWnRaExp=iz79|#eUsMhT2Pj5gdm6 z0*Zz{2;2EE4Tn6+=F#H3r5jkvQX2!Z3EG-lw}~d=Hz(6Y(iieW(Z$fP4S5P4kK?<_ z$@Z4l%N0a^&@A-g@zL*JvP7)x;}ArFxDw|m-t<#F?yKgjlnC;)a;L=GxWjat;s}Mz z@eAqZ+CWaEvlIuumrImJ_>H7y3q*vokWU^1Pf1M!;Z&~O#BcDsad5a{;fm>TKtde- zA&L}_*dCaIZs~dA2E|QLb6Grnd9vuxpc{8Y95uUV?h<)~)ZKuk+NAj?s~t&JN* zRNuC&{>_<{qYcWM>JHVrodQX#zB=nWH1Tc%dMX5@j0Y8ZxL{ zZLX@srQy(Aob(Bab9-LW2B@57b~knjEb!8*@R{lSCkg|k0mGQzNj9$@O};%z_lPF# zx6T|C4RtCt%*yIsNYD(luIWFj&Q$JD>R5G1>X^-{Qw+5Y31+l876QyUs2c3Kh9&Ae zwZg(t?bDAFil1*HsAs4c(5F{)t$h{j8Ih57dV|tj%LgXwTXuO3BezipaLW1yG>X&| zm?qGMZL%XkAp&O(#o`bvcNH-ZddRC9kxB#DRnn*w-f){}0Ldf~yR==^2eMs-9U@cL z2u_jdFtO}mw|Lm`7i%F9dMdy_zq#u!Qif1sC>`)@g6}2$fG3Cq2Qzd+7va3cx zp%EZ0SmQB|S}D$-Me#c47M-WYhYrf|wGx#dFIwJMqk2|ZnmCVcvqM>Mx7{Pbsxv^- zDTGm-_2Jw*4ol-sPpmS9w#HDIKLQswd~YwGD^4I*c-V9;2?~s#K?pAa7m2HQS2X!n zNM2H__`DkZL34$qIMjP0kmWrw7LTzqM)oA0TwEx)IsKy~Icb3o$mx^gP>?n7-^{W9^Gi2L(owCA4fZ9ni&$tMxrodUHogy;N_A%F^~cOQe?(vHIt} z+MQ?}U=-+~eoUO@qGpM|)#*(IHn-@#)nH5>xCP9dGWr{008#W0RT|{)0>NkDu~L7h{_2|iYi~}YC3JSHTcfd_6}5!x5-I} zwKy^DWgd{$i|dg&GI2gIfyT#yWRhUSB~^dEZ>&nh5s^x_kgCfnp=yF)Dfnr0Vn>&p z5T4Ba=y~9E9CM7B%awwJkdR7BB={%5)FY0Qvj|g~v6GAvGL!R^Ktj&_Zp2^;r@%X) zlvIF%jEZ#-@)P>DH^sA;)7*#ZAUAkkVjXjIE`EE1!TIC)bPnQvH+K=l&;@3zU${f& zk@KG8=H_;?&wzrk=|99>i{Pk`;uL5^ULY)*3b>bcocN2voJaOX)tDP%ev5!8&5W{i zY;_1S#*`D5Nni1eJ?*3O8vNzYEi}ZFg+M~o>nXudUvSz9`ziJwW)tYT%T57TV!~_) z*aMMhmdDq^z(HhRJ@BBdZ|+SJgF|4%A)BF_WOazXw6otWrf?7>Tt4*rF0L6&M;sV1 z+xQIw`8F*|9KIZxG-Z=WqDjVNE5)Z_d@$nOOF;b=*Dmh~L%*ol>_}g{q$nyrOsLv0 z6dp?+P_t1)g&bF@WNtKmD8R7M;4rRtrObfi%$C#6vStr~8|~(~@7XW+w2&BI`vu7g zcRp^OZXs~Ve#bR@%ts{&cO!(PB{NP*a)v1#=yh2EWpH&?3Ym;N8iKM;P z(-j@ygABigHF+qK{y4qcAMl?zivP zK?N4}+7X)mI+_Tw6`rykeq)Dm$rR47Kl1CZ5(bP|^$js>}e6`(sEUeMb1 zNAV0}2NH~)9v_$y&91)Bg$R)_ov0q~vANp z?u}x&$GzY3xn2Gkk<`qrqaMEkjnu8;GRbMB%Qzr%BIAs3jO_s)!V2p9D9fB6t3{E& z;U2W6YahQYvGZcMZtYwk3_`{DW@Z8yCWLEh2p2DaRN)HmX8x&o@Vx$bJ9nGfMq_o0 z5Zzd}wb33(%B-Jq--rST{2k+1E7lf7;p7sdKN0`g%uZ+kI5Ok&y-+w_S%3VzwzM)e$WMZhdu~|1Y6K4G-v|`fv2X>HZP+1x4$WKTu(MX z+BAz^gejI?nP15(F(B(*>%{}EGlG9ZnL@U`M*2{Mt{P41N|i*C3^Y9Eog|E?gZy∾}cN8X8rpJxFIYqxx6hbf7F)N%s8tggi{ZQj`DH@>&*+{~s zIC3dDujUlM7~CwSqINI`8?1O8fIF9zH8nlScjsYElI5ZnpM?7~lI>)&HNo2jx)FZN z+`#3l%z}n~AGg~2>)c$uR&(900^_d{tY1aj>t-KjCL?5rJqY(cfe)G$Vz9uJ(TPbW zEN0%vG8!q+u$oOVsngGPD1p|85#aj1X}l9(4D@<@TG*8xN90DW!9@f~5Uguak4#Io zJ;ixyH|-2w&+HX`0`P-*?VP{3OA+^$KLIJzlF!5dgI)49L7b@rWGPgUMCmndHk!)KaC8G+eoZNC@#Mts3 zo>49*B67lQkV@aIw_F+}47x%ij$oW~wZ%8|mlA95qle4_mJT7q7GC}ozdD*)DSJ3X z8t#wIzwj@$(68u^wNF3Y_*u$fL)|5BD$F59GRnqXTa%$#DX4QUcpYocK2ibS47vPB z4|HW7BBpj^RS7EP`8*a&IASHbD#`n9xU~@}U5>N-$ti4ITQ1*dmsaIh-bs`~i01U` z7w}X@+l;B1n|01@xbrnx1PrjGp`BObhXSfSx$bwQ`txz#*i`6GaPM0v9bfa#42LOg zg{I_XmNROt|CP>ca}Brw-MndL&;+1s2^bdFO}h+745f|9g%Qh%X`~^Dc8e*oUhP07 z8ZUY&8E8}g@^)%$pw*Z;BFQn{vKe#2@g>wxL@=R^^2|mj$vUBJ`zzB)LtSk@Gx5sF zXE;ITxlNc^c+Ia=Wlxi31Y`kG7==uhB&IB_HKqaPR%HfZQWFr z3zrD}|Yh@xTFz>rdaS4DPHL&W$Mt=bfe7%8U)0P$r^@a|$Yk9-V^jq`RAdR}#9WZ)DPylpojJrG^j{xIb0L z%L>^)s|fep8Nw6P*FlicM5SD%mbbMQypkocV=4*y$uvu=G0Gz@vMT%F%llH%DWxYB zI*%F&{$gFy!izJ)ty)ikOofhb?Rj@~qqA#AyRZCae!A-Zp)B}leY7-y`|6bidPLH_ zY&nQ0$8_v}vjxVI)KR*zCPwFJSU9>tOwOt&+KC&wf0FB1 zXs^4=`nskw$TK}vkUXF6oKxEgn)sXttao7cbBxvd@iP?ol?#w-PBKzTVzMK7O@6ks&C z4xD+$vK?tSy5aHpy0-j2Z0~}Tj%!cSlGlZ;w#qfrk9?2t5D?{B=k&jGOk<+^%78@wMW~kqw?f!pzhV$>bGyKXmwo1 zl!OyFk+pWp6RZo-;$~O-4%eRgUKRauo%dUj!%B?%CB7<>|S*-vn%ikANwHTz(z`v{*+VG2t)0_08c;vtbp7ok(lq^e=i6M| zFU^e>0bXYu&t|%&Y#ZtKWx{erSC3fpiCvvy`BT=0V+tD#W%PiB<2i7GrqV66JWMtr zjG_Iu`?HSe#aE#BtIOSw)Bs|fWjv4>#Sv$MZW#*-A>zdaQC?kI0?JR1_X@=0PYt$p zu=&m_$aMj(_Vh_rtU@*Wi|W9pO02-;H;sn}_f;D8A$0g~S38a)wc7->)&|-dnY>^# zyWo^K7;I5H*N*j;7KL-H!&l!{M^B5FB$k475?kx1;?3l6nU5mBnX7hMH*tJ?wX)3_ zL1~OpAiH36AIrDSXaeaKRhvu1^aKi=ZIZ>l;Q)B^O5+u(l1JK3e<0g$4~P=t-5br`2nuZbz%OT3zJGnDkkUb z?O*GOM~pjGF>Vf=IA7xCWed8f5&4S(h!*viE;_988iA$HArC(S(XjYw@=U+vx7Ipg z>WxGDP|Y}@HG);r3dNF$&uB1_IyI;szq4Q?6=~_>Bp;_zwX6$uUv$+Ks*_n-rsrzw zdIU=)cU=jW_xP;OeDJb7?T6FtK{6txNE5ys)bPMBTc*y;^rojV1pW3X2-mP5(@C-l ztmO6$|NcFN+}^fbLg5pV55-v=>X$ufh4z3PwE}Kt%H$p;vcGX>NCU6&mXVIr2l$El z_T7@?PDU1DjB;ubv4#n`r@a)v^Xnerh_{-0fAODo<>91xcEbq~eLq!ES^JRg=Uav8 zal8{YudwFMx5~SbF^UASQ}UCUrbp5dL6RHQ;;2>~LKCJx8F;xr3fOt;1F0Wf$rp)r zd0w~rZ-1t|5YK6ti;lZIT5q1o(EKDWt@Ou$yPmyh9B+u5lPGH(;tKRgZRVMb#rfB(4 zzR<)!`-2m0YQ&)vL_k23QMjEF%L-*g(NZ6O)gjSE+go{bQ`5gGou9E1Q{#>iC+Jn~ zuA|{g{;vFa{61@5{o2sBPCg$;L*}KILHby`Se`4of5b;Fm-KcU?Cw1xRd@pSRjun- zX*%g~+u=H?R?o~*E$+qg%WfGjhKWeX+`S!UOR60&qU(a>D<)XRyD+89thj7+uz)v` z&RIlDe?PPv&qOMaLUB!_AVPHZAt{>5fc*h%sCX0n)5-Ngr9YB<>=JT$FD}@Q)}v8J z0BLa+4}q1uL{}3r>QLR9ga;0&wjh2BQ;aRcXULry@O*LvkW~BL``*VS!Ms;FMD-`K zEFM&F*{~nqYL>tLSFfb=ocT|Q65UauAF@bHp4($#jOP;1yS^_%`}~l>*xL%1x^ zs09ZZyg}>ltSnkXSH8UMh|+1Pm$)d%p^Hr>uaZUPqG&fsXUEuOdYfVMs3lf^Y6$0qLn;@3}>3YZP`xgAITM^(f36mKrB?CQJS{ zC6%ba@;<3A;y(IfaifpeVAe;)JefpUq%5Xv()X~4+udy@=U1KGZs*hxDt90E-N025 zRfJAooLJPe7Dm|L{LI#a=^_K9N49R^%u?ce$*}qttPSeJQs{gfr*(FjX36?7qA(HR zOu7{Sv4~GsH}7muVl#QpOQb(EBz;66lwoHx)e0#BcEU(AzF(4I-difGsZirx5=+aZ zsG6|i>F2G*=?IL_wKDv`WeXFiTwzSmh}A%mO2|32ymbS}j&`fpMppWkhB^|eXtnb& z+Fs_(uKW*+43qkC-iS9x(o7_+w}%@$PnrvC$B%tU>T}wRJgZ(o-FlHc`o1bB0UMg3 zBon3TVU_h)vn=kIRk;%qznX4ktJ^v z`0h;8P#WHAW;Ch_i~~s%NM+P$)Xwy1XGIoXgb_!C1UM1m2^wBfeqdkM4g^jN!i(+(1K`OErY~A4E74 zvdA`y8YE^JAu=R5H0Uo2B<5kTIV7XAs31}gH;^fv_2?GHNRn}EvWRcF7_n3??QF^~==*Dil?eN}sy<`|%s_)T+5^ZjbxKpAb%Ny!f{mz^R zmn&8EcFa0-_29~0LU)k39r7a;p6TYdWClKidcX&Tn9oZ_t@q*7D0Ulj=h|5D3xNY0 zl1_HyCK3)Qc9`=}r8&Afx-#OF6VMD3CDX=+n`7-l5Fn2VYEn{Tg&)@bZAudPCyKKW zc&%oGk!$v9;xwi4cy5 z$tRHgr1Gtads3=pHYG7fjaEb>CqzDBiD|+G(U2$QS}et;SmxnGdD5l&OEuH%=%E65 zdQI2H15wo=UhBdtJ}))?U5VJqoBNLng1Yb5MaE`sV%lALlf&9H6fVIpNnNkFc|~@b zNOP}MI*FW_WuVmG3DR?t710A9U>zbJ!#w~iD)TNu3U8Ule=O9b#GG~PCtMx^T1YE5 znSc{TEq_VSc#Lg32~=0@z-xarv3dhC{hfM-O8hDwv>DcReRD*+-mw60LM*{N@cr?M z-EoaU>61BtP(HjwPlPQoR%8NaCQ*um*=1J2(Wgw3qm7_kzu#YiulC2A{p4H_Mp{#O zagLmgX0Ez{wk3Qfm~Tsc1qQImMYolcuypWfd7*uVpdz%#;Lr&suB$v`(&{>hiKwuZ zpo!HYwiwC=8%k@sS0jM_CtU9h~^&To}*7Z|{LcnJ&Vc!rz%?M-jt%kFZ;|SKJwZZqTaCG@=E`mrN znPNe3VCcLKw92aw_;3=mbMJs=lC{8zgp z$dTgP9dIgTv$RHN(zqM1WO8oA_3=2}t_NyNw2y5N=2$yPA(`e_)G&T^1gAzqRdL`s zm7RlN2fMxtG2BFM1^=Q-tKNQU^*^{r^{ci{ZO=URQRs$k z&$fF@IZ(@~e8IQ-y%XT><-i#P?+{a%epE%PZ@cNyRiNinr5A>CTJU^?<1o+L#?gl5 z-P2_&N`$duVKum51#A#|hni-$Ik(Ppu#e;%9K%=B_j#p)LESRNPlrS?y0XhZFWL&d z8+h1XIoxA;XAcohRmI+N1%xRRF-L*4%phk^b9?Ib9NU{*&eaNYnR3`?42zOZ(`It~ zb6Gp^7Hi&Vt$*Oq0=C16vO#yCp9mAXjCmhI(xBaw=^UionE!>|SgF4~6n;*b` z7tH^GEq8!yBU^z00Q&wymstM|Taq|?*qazRnix9UIWm$k{tLy7R?(J67DV}$vxBzD zS3sl+q^MFVQ@GI8Qm9IcYcf~b9YQZP2_W6rGIdQY`jOISc-@7+6~~;h#uhfQ+6?35 zKDp0k-pOL-_w)Gy+@*AdPjkElcVi>qVrg?|%w5YXJfcfOqJw6sjBW3S-_H0Zz6^k3*>8LeEFg(lz&E=!H^K#t;r*IOFV7 z;Apv!{4hE!vfpOx6@+OXRb*jv6?gMT0uhj>SKH^)Z-WQOzJ1U zB72we$!TF+fsMNkC1JPKMkovhyL;wZy>`mMJ?xWCy9ZVbz4ow|a)1feIZtQHP`TTc zQJ5L=Y+f(~dT9@Y`#fVL#dz1p=E?~ax<(cH^K!#G=zXfYMB5;7e=0#izg6IYzp6Eu z6^z4iNAQUC5<>v_=c_0xT+W_H9C;Y$vV#*SMC)CKNmgx0wq*QJB~{+265xpXAHxGH z^A_is{8*e3Y(4b$K!^zxvUWSBv|VwgiF>e#%rrqdu~k#As7mR58-Hx^FFORL|`HUheQ4xVfar%8{KVRQFF?^`}+kA;uP-0vQb4BDr)fkCId$w(>VkbelKW6Fty_rkkh z+@n5F^(3p6Wb%(JH!K(O!7svDmfNiRqPQEbL?L^G`q6j)U2$Z5g)#6-&#Z**#R72q z|1rx>zUx-?zcd1bzgQOHKW6D6Y-eXJ{12PpU)ETyexZ!Lg3AALT~o9KT#sZhI4~V( z9TXQw+Dl-VRTT^n;Er5}W7ObJvJBCElr|_g@AlEH&vQ_V%a!dd!5y7wVa9cq>RR%n zXqM4&Qe(X4-#;s3p!Raj`;zt0dGbBo?)LMv^XC&5Kx~g4geG!i&YKT=s^X{t5h7q& z{M=|MSJesg=l;5IQEvzW2THBwtfnI=W?ye7-JzM0*H7FnrUMe!mDpy5kf0u}P!3z(pLi+I@=RX!q;1)&bO zpRS5FNW6liGYplGq8drO2wIsk1r{fv)L_~wofDy?D`Z3#nJL|%a!oG8SDv?Lm7%mc}Q?V$X7`$fS#U0O+*dCL1}$D)^YoQde~xHF`aR>ec{&tas_v9;g8ge9&HTd>k)ot_VI+|m z&)oew`vecwEx2nkSV+xuH+!~xZA~SKLd5#PO{%ZC<>gayfM#;9fgPEUBl>LOaQK zaFc|t9xm8P1dr7|P&&;eL&GBp=!3_!tQNGX)^o7FR%ilwKf?{e?++4)e#_X3M{t@X zqZ}y+9PS6-OJy?Q(v(@aEv~S3cwsot=;}AkX5xV}+PKc<|1QF_#Vp?;M!Z2|#1Hj~ z8h8m>104yivgS1JR2Vz-By(%VmS-No*?P_#YCZxNw8=l}OR1jd%@`5z2r??Q5IsfUqpWApz~nvIXsL%KDA7IQ}7ts%dXSKrJ;8wjcPR=rMun{@3Cmu zp&Qrc>&F!xaI<}mSEPc|V*W1bJL&H8!;S z>N_l;P*A&`SpZ?BQA8jmT>CV+qCMx%nPu`bV$#kbEp29BCuaM3`kE5Za`@|h1ZA^! zi8>g2W9soqMf*Co$tg%p2w%?H{2^SXW2ndRCEZz>;`ttPFhKT;dV1{qI~}L;Hr6W`PN5p zcUe?j1(CR(^u?bavud4<`k*DuDCCM1E8c_}7dNq}0PyJCsXy_BMz%fTA6FCi%wl5#!#m#-u zQl87^N{G9dxq>9sfLvx-vF@V#R9C&rVSmCD`l4fqKk?o1s77s4lMa~4un?uH+>wH{ zgt}((E-G)FAIOgY$yVDAKYPQ>fUYH!M zYE;iXszz^EBXz^bP4ZnFciiA~OWW8GLjPCz>DX_HInIz%x?c(Tv4;I|7%vsY3Bv+D z>yuv|5B$?f3;x&VdA5rJum;{x&6$!knOs84wB_RNsRB9OU~Q>04OZY-FC*0D30h;R z488%8hq3|;(@m^*!HRcqByzg(Wc2sep!hJ_y#T&41>*ss3O1!^zikoLIe*mZ1S26( z#HtNypKP`Bo0l^Mw7uSd?q}`ACoGaXL4^2|;xg6Jg1JqbxI#F4t2$nkEt)f`+kjyP zA&|S>)KHAeoE0Cd%!B(DMVZ@{Jt5R@naX$>>$OXp`HGU7plbZFF`^2?7xKww`4IfL zshU45Xk&pXs&^PF+ap|)`kTv8ZQ$vfC(`n@`2J)z^i$Df7=kWCAT-(4ZuoW6|9}&o z*I*KM%(R)D;?w@-wtJ6p zf<$QTC+097MXV`Ls9iJAvU~qJStxQ~qkGYDNr)}m`x;a!(a zW?L1cy@ty9J)^i}2rWIky4-eW}uqnY*WrJx#uv z@eQJzlREoATK8w>E`K*T*IN*1Df7=6+S{8;XVV z@W}_(Ei?0Wv1CcM4v!dY&MOK_<%ZGGe#$kjw^J7256F?DKe23bDSau9RRxb1=TUoT zLv76@=RSJ5BkYogMp>ojMaAOn5NJkIe!BnA`?q(D9gU>WmNKtjuPcV7AI&D6_jcb2 zS>yqvbHnc}|G{NfpyO*hgD;RadKU?PGj5=Mzq~LEm{3e&+ha`_+>R*)qafir5NwKC zvzvsT@Ei67vRASret}qcdZ_R-m2K;N47B-BuQJ;{vn-|ZKVwd^?6K-uJhm<}cI!Dn0_3{{Yd#HF| zq3pn(Hvn~B2Hqj)I0u?@l;!2&O^kax20pP}{Q`6AH2ydKM#}p21}h`CU~5@W^jIqH zb5VKjW7Wuy!e7ClGOPKu29pP0P{X`EwFHN6SbW38rwjaSTkBeY*VY_o)+oEvVu075 z-$0WC^v;b$pGX?T!y7<+DNIVo>KwOmb&c7Ei%!b?gpUc;se0YP=f?9hxY^6^Yo)n{ zp7{$_DLEnUV-@O9-R|c&q56&0s1%>@|IQZvA!Iw&&^(C!jlM;wbxd<(xPML$kYdn;2v zMK84?O(P*QEwMyRCqXlQEJZ^#CL=9F2RNQwgqC`Wb%u%M1PJi&Z~G^4H+%v}%K-)e zfc(p>M*5#Kg#VXzoULjjk1dYiyKI2Zm)l2HTa=$4E@>TgfFQ9jNC$gFN?IRq43W%F zXG)lggaelcy)(|dAEbzg`%>FRh$b8}V{dFJ@0R0-$ zSB6j9S<`L(;L2SZ!1$?W#oj(Op||_GoKHc2)#4M&{?*)T@t$%qscuF8W~JBC?xeux zGvf$HrB8AeuA&(QL(^&vHM8IHguT-&8nUjJe;n)DVlI_FzU_J3E5cZgbZnpWe(fhX!|r^bvsXJ_tDEZNE&w&2%M+ ztA58i@S6A8)Emx}KKUX@C8FB6me4Aavd`kf%4GvitDV)Hhjx}*b0vEXcZc`BVUH`Q z$}<&L^CUrak7P&cfb0RrtCDt2SkEeqI}0s3P!#}~m58&+W>31g#dxdQ*69_vHMNym zaHifPg4EENR~YOCgAV2nNqc=kvCo|@_>+v?4PqIWpwByjJ|i-5EbGufLYxc3{()U_ zeGS@j`W)@ta@1}Y!}3X>`{@3gPA))e{tkkliHFnbJ}7QTZ)WVQChHiJiQG4 zui9R^ba~eP~u2eVxV8UBXkD)KR!EoEgdV^pzKlco}W9*0})xK+%Y2x$oZM z@GSph5-GkFDSm}Oj)ZzGeP3TAw^*^@4pLsekYQseeTd3ZDt8F$>X+q9nyW69I^+XZ z^azw!hc?mC>F@zF?&3}^5I3GEcFZb9e~Xs*ZPxO|BXv%kt%+n+S%*PG&7uwC80tVw z@cV!8R{yc)hi?{**S~Ea`hQDtf4e3At9#&IiN2_!t{svhLiXC$gyS0(wZ5&4Wh-qx zX?bvwhW4OwzJdjzb7Y*#(Vh^x4p8S^wNW_bvEqm^U*V$#AC3bXK~PSpxH) z@6T6AfYLl-3f#RmDgq%#GQRMAowkKvZY+%>0Z>dr#tVKuIFaoFt2TCt1$OesBoxnV zRVOmKh*jR5fxTiEy?R8SaP^-dA*|`rVq;7f!Q>?-Y;lsf3;kq{S3+H zQ_x4UcV{_)B_&sx6(~ocs%`XMGaT%sWTa6{=*?VYM^WGRBK+9J9QaEL<*?*uf+x5Y z7v6Ewm)9o7LC3y%?QR(y3hkf9t4s>86J39b_@I*yFwoUU&sS=vU9pt*y|@{K< z78UP$D(dNr9blWg`rh1skl>kg^s&lRu2ynuJ6dIWI8T6>(G%64S=5;qMr2kpw+z=V z&{!(-<8qw$^X$kSj(#ZY}-m}X??GZk|P)p1=fKZeOgwZ5IVi-jf zlf)@dNJR)8QHe_j@%xW@(4pgBZ?EFeEeQeuVEMOchVlP;Cvvy3&Q{&9-QYm*c~PH) z@-PpO7%zs8tSacwg#Kl12_D}ln**hVHDOJnL|DGMo%eCYT|}ZSr`ND}O^5^XvU{}BGm=Y=)69dvTfT2o5 z$}c=ClT;ZZwE+#283YVJbnHkxkCZA7NRW!6SO`2($`B=YDU>Y|3`wH~K^jF2900Ol z0AIeOun{`#0f_LR!=d0-+7=CNjHWHLSS6N$oF>I&6U+?MUJ%C| z1h5??=_yb>U?6?nIAJ0t+SXrJN9sv%*@mpUXhZ9AU`O_Cmx9Ay1HUSM1dHXo_+R|(vM3c}!jVK~q4B^@WByq$Q~sD2XI*F77|Pkf zdyefeX~eh^$zi%Nd#p>YSvs$@4ks?;P>UH-&nix$$vM>ls)IBp%=_AP#WqD3$;g0+ znp73-qc4K3RQ4QSwl>l$b48yYU@AOhz1U<7QarCtX;BG2VfQX9-5Hy*dl$FQt{gcr zbVzBN{cFbFd>JuhMq0Z4_MZTL(ZeSn{wf?$!Ie^LIKL@cMTe@+9LOtTf>on*FQAn9 zPAE;t?=y}d+KZ?Os|=`ANLO0AGbl{qN_R=w8`*HT^0>N+gM)|deWhLm<47Mn_Mf<{WE8_rqsxtb}qHJU|J%?B{l6QdtVEiX|-LbgQa zjqNf*btVqSP!62Xur}>Ye}NrsDB8Q@gmf9$GR*_n*+hHz;W+ z(B|BQri4aB?{q{ax^e7=|~6r2%HNK)Ool~nn)HiZQyW5|!8 z+iEfDC^k_ze=O&ziEgs>nrzw$psXvS3ro)D@0D$2Jx+pDcL9Z&yov5*UXc~&v~G>o z2`N=ew7Row8>X2!BVT3L;p~N$mtw){V!6b~%}qNVbj4`zeObEgT-5qf^;~tICZW=# zIrJ@ZnPwIHk_67XuLY4U2D;auDD=Y@{2P za;{Q6o`*yJB$svP{foB?p-)-Y!Y1&$r;C*xo_2F~n6FIr+UJiioq0AVqt;ri?eT8U z0O*_A8(*!+p>wIOZcSaYER25 zM&bJCdMJF_7VH|`n0H=;*=6?9s^L?DRjOk`JwR{EQ5)H>a%ot6=I+x;MwK0V8uUi; zDAclkb)KEr!E0{gPzsP}=X(T)9G6i`KP6vXSx(fU**GB99nDB+uG*fX8-&UNHYUV1 zl8@m?Mq%J2SIKdtl8{>dn7f+v!@a`3WV&V?_Li+bgjp{7`FPgVVRQcTB zZiHaOQjl^ZCtWDt;WP%K)ZFc=r#zc0(MQwNYQMka=I;7Fnrcbb+D)FZ3L%Sh%x8l) z4_rIMC7WydGP)Z`BYIw+pKj*7Q z6DR!G4rFwCwLVj`b&vJJerCzwJFpZ}P(8fe0nZVemf%nUyS1-yKzW8R$pPDsNEVmt zv>_=W&X(}Rwd_=@t3r+*tz^-Dg0-7U=IvP|HBoV@IvX&@P~w6Mn1jrnD;?Z!3z?tl zMs6LO2sw&@F2Xj61@&O~6@ou&pIn^RWK5^TOtSR^{t+Zb2 zH^<%lB@o+Kku?X|LNkb4(2YVCigt$HB!RBG8 zag-D!L3km(I?0lRQYT~Q0^XVR{)rL`EVq>SNIzKr*7d}Y(S0M)Rg0z8tTghe(%pnY zX^ze%R#{~U7YDvY&a??u0D4fASlt05w*ny2l0u3aPE;8`S4J+C?~+vhmvcgt^nvBK zd@#a^pv<;Gk2OoKQ&y}9kHU5bIb?s7E)i$E%GLzs>nHU_Bpm^$xYN5V>u_tw4u;MyKq;uD)ZYP0Bvldr4e83S#%D?$Z*?R_&xDaKB4}#U zgl)g+=&W*NF_LmSzc3mb>|V}NK4K7jrYX$Ruc%WyF&71w(3<>=ER-vINufY@^w>gt z8KxkDA8$A#MijVY4f>oyG_&+}$r2L#>cZnX+B>6^v+4gk_G+%=7x=g}${@0Wn5YDt z)X7H?iSv0Y;(lqLjWqD{v14>FyEH#oLG8)@-)2Mmjn=!@0L}2H(V?Z0^EFd4=|-Gn zR29j1ObLQu|LMW~-{=6G!OB1b7fS#skbB^>D=J>mI)%(z()V3K$Oj{-y`Gn7*yK)e zQ}*tf+Mu~C2%X@{L~*$)PCGw)q4^tusBuQT!*`JLq#QRRuvMVcLV3^8!2+>6{?j^^ zxIPErqT8)UEd0M1B>Z*+Q@mf+J2W*uPyhyns*unVN; z9yFP60Cm9)x!>}BAAyFH#j?3X^T0>=TLSpL`2i|M;jS_wg7YXoSH&5SWx>ey@YKL5@{pJ2VDey2xqCu4!ruLUVv`^Sr@$xp2N5rO z#pzX)H5kgaYfj8!;ygi!LH(RVQU+y|87hg#pinj!^C<}yBR)!__XtpRaOp)6y$R8X zr$cCg>VfG)uZEtlldr;n>Pe5}PL>ZnZgQc9hg^$Mgs@itw`J}tG}FLq2eBZwW&;{k z#33vU_+hUnlHV|;+mk2IjMpM$l7OmKqkML>!8#3Z>jA&?L_WU^={ih|L9ozXL|;1a z1>UUK`wgMR`4qiH_??ZTqU6JIP0~}!g!Q~>>~+6Orf6|lSm$?=A2Wmy_$=s#LB7e` zVeJ*%hA~fpto=fZDO{g*Hf_?Io*{#(}`gX#I^h&e81Sa4lJ|H(iFap5~1j8C6Rjk}nDM zDUI@UQL~YVLNe>Y^cPPVBGawlDa6oU)-JNscE8z`R38`<7AbCv@>|k2z2q9yAOxEa z@?w(Ui1CGSzNAXKnae95_=9BJkq0lX0HiK?R$oH9Eb6UFPyJANvdOOJTGGS|m|2wy zz6lf}#3{g(<}pVy#i9k><6x!;cxv?)9Fvy!vO6wj>8~4BG%eb zg_85vO`<_A8PkV)>(PpSKkZAm``?rQPbF6#%~sxqt+CV+wTs45Ra&BIU&db4Uem=` zikb>3ja_L*#a3ldweM>CFxFDFC6?ONnrdrE5e-Ta(<(|B`Ls6KDnis^}4Z2d;2y>%#&(brY;#t`XQHzs}VG)er zbBL(v)x(@wNK0jSv!=ZJ=A1d!CR5?k;+e?0(W-2w){*J6-t^3C^F3F(;xqMMsX8`5 z`49rjSqv0(g+ zmro>fSPsNkuw|BV+0!`+{0vzG$u><>OIpd+>Des2mmsif=oX*+F|PE5aCWegKu2kL zEuzwA^6q9+uqHJZfCdNTkptWD>kJ|#{rygcMQrfGOzhhgBHTQ0^YF8pp%Q;-O+)W7 zs&4Vk-Pauk#OM|jD2|m*=`X|Ms&dr`Ovyf8l3YO;+`2rB_A21*rFLr6*&7&S!tV)N zZaqcQO7^USiBxqjbLMeIYOpNAXdRCb@?MA?n=DgaX$GxQ)yn{<=p={WX2SvF^bqb| zM~w|T%XJ#x&c|(@(dn8ov;7aeWjrhBA8Oi^b$b24sVViU-}l@~H=Y=$B(&1m-HFBC z@NH?n9-Yy&W4$|@2B{Qb^x45YY(hA=Ch=wBsa-z_MBcFY6@2v^;Y#@ z9SM)K*g4fPFjuJV?!%_c0aj{#>KFE5Fj2Lxmt}D`JE^F8>%x+&Tz~aOiVA{Fyz|29 z)5(Ph$WV$^0R3>zZQ&oR0En^SDo&p;Gd0lDunZ~2ToECDAJ~M8gMDG66wcjWmLK2M zI|Q92hCHAd`~*UY=os|5dxAMurQ4Hf(a7lqza2!-xj?f%2DY0a{Q{<-?FUp1SL?jg zvSwx%2}W9hDy{3xtFLM%poS8{b%>;3x(I0F9g+EPKv-8AXdz}6*6%-O)#+5;Wivag zpHilx|119%2pa%(u1_~-fn=jN*{x@)LC>k16pDUd63;mi6q4umfuo;j!qQIa6RWee3kVavSI@12WVzl8$%>IDfOlv9Sy+Z};fc?mBA^6+DdK z70NrrPr%wr8I>`)gaJp@7+^(TMK)RZ-tgIk@uLHLGhwD1bqW%x(BgJYI7u z_n50SRb(KcQs!Od$_%`qPvh;WwkaYnL}KR_7sgd5cMR~PO9!M8R-zxTx@G-akrxd! zotRqaE&wjctvB}}#U9GB>2iMNH>?O9=i}1VFHg(@A&?4_jr*Hut>k>|%~c4!@6yX6 z)m&a6lh#^37_HXR=jW#s-GE~Yo{w7-O-v8GO65VrqsS&xyy>cWnpMJV$ZJw7QN#&f z7wzIK6y;fOjV=Dv$BcalFcpMkBE976^;xnV%TwPXzNFY&m@=xLSe-@T+fla!hYCdQ z*W%gWWh#85H*qoyo7-1*bOJ-O4W5;?jLvEpmX<-&Bh2GlfM*~CC&g?~I;D_NIuV+iH-1zf)k`-OP?a?vUK_Td~H zcsEOqDBYd1Oc&M;7Qym~SfGa%Q?8UxH~jOK z>qXqs$|u{8yd-BQH4PW--{%SK|DXSN6(C8c?_w53y3hf`-^cYF;DJ=j+Ow1l{{$ns z><$(Q#=Jk{{$BW7m=CGzqiYI(!|o3kmbI1s3jT`(_fM^m;(ZPliOu)_U?U59lzU4W z*dLDj`Nmv;M3H#O;XjnXqY^5}KwSoky`jvjprjzxKJ3{gQ}}?t);`j~OUZMT8IsK7 zBtp^3fi;p6Olsn=Ly-(VCb|zk>{q0OA0@4l;qlTGwg08iN?C=} zOb_D^%Q9sA_Wl%}^p+1q8cN(z<9ad!)pZhv@BNq^MTjZ9y&FH?X2zP`@jER5*^ Rq&wL658izZ!OW0!^(WL&1BL(q From e221b56be7b6d9e48e107fc4d1cf0c15f02700f8 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Thu, 11 Dec 2025 14:00:55 +0000 Subject: [PATCH 291/400] Preparing Spark release v4.1.0-rc3 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils-java/pom.xml | 2 +- common/utils/pom.xml | 2 +- common/variant/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/profiler/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/connect/client/jdbc/pom.xml | 2 +- sql/connect/client/jvm/pom.xml | 2 +- sql/connect/common/pom.xml | 2 +- sql/connect/server/pom.xml | 2 +- sql/connect/shims/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/pipelines/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 0b0b67f74f48b..4393175430265 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 4.1.1 +Version: 4.1.0 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 0bd7673d60185..743f74753b1f8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f26b72b08efec..1afbf81679906 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5a4dc11f51fcc..167582598a920 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 794b8e9c6872e..8cede6cbf288e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 21507aa76efaf..d17a0876fe420 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8e51d311be5f4..161802d43ca89 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index cc36b54df8519..5e851791a6975 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 130153d6127a7..869931e403349 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml index 114086da25cb3..dbac5e4552df5 100644 --- a/common/utils-java/pom.xml +++ b/common/utils-java/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 981d484451865..1805f8462c278 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/common/variant/pom.xml b/common/variant/pom.xml index 4ff99df6350fc..adecb4b4185d3 100644 --- a/common/variant/pom.xml +++ b/common/variant/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index f6aa500d5f007..bad1f7529c054 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index dc01425a617fe..b5af1a093be36 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 4f1eb8302aebc..0022adff1a208 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index f6b7dbcdc3c2a..b482aeb77a96d 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index 758ea445746df..5da0ee078e26e 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index cee71d4ccf453..2479590341822 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 57694e27cecaa..9acc9b65d7653 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index 91f278500d78f..54d5747e78aef 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/profiler/pom.xml b/connector/profiler/pom.xml index 356934c9eb929..9d486a868c3de 100644 --- a/connector/profiler/pom.xml +++ b/connector/profiler/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 7abbc76899879..a78b722eb6bf0 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 545017f1d77d1..7ee7e096d38c8 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2719881e25cf4..2281f3e197f88 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index bda945e03c203..dc3890e4522ee 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala. -SPARK_VERSION: 4.1.1-SNAPSHOT -SPARK_VERSION_SHORT: 4.1.1 +SPARK_VERSION: 4.1.0 +SPARK_VERSION_SHORT: 4.1.0 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.17" SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK @@ -39,7 +39,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:4.1.1"] + 'facetFilters': ["version:4.1.0"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index 38daf00501ce6..cd17565a1d98b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cc4327ccff090..602817ca792e4 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 66b8cb104e07f..fc593ff2c6196 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index aa8812cd9a1f6..e49a0678d7182 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 34df7e4b5be1d..6695e77228935 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9061edd455ebf..b31c2b7fb3a24 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 1e61d457e20e9..1824a28614bd3 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 3080cb455888b..9b0e7c1f5ce40 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "4.1.1.dev0" +__version__: str = "4.1.0" diff --git a/repl/pom.xml b/repl/pom.xml index 74199753f2c5e..53bc41d397d60 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 3e58d34fc1857..c5f2749ea9fb0 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 6051496bf7fe6..3b471a8ff4133 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index f1a8c05034f60..6d0d0305ef710 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 7d33b1e5f107e..e7997cf61a457 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index d7b0d7e0554bb..8eef00489669d 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/connect/client/jdbc/pom.xml b/sql/connect/client/jdbc/pom.xml index 6dd373071f6c2..07d7096ea255f 100644 --- a/sql/connect/client/jdbc/pom.xml +++ b/sql/connect/client/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../../pom.xml diff --git a/sql/connect/client/jvm/pom.xml b/sql/connect/client/jvm/pom.xml index 40621d51d7dcb..6532afa46710a 100644 --- a/sql/connect/client/jvm/pom.xml +++ b/sql/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../../pom.xml diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml index ecd735cb81ca5..3b74dca497387 100644 --- a/sql/connect/common/pom.xml +++ b/sql/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/sql/connect/server/pom.xml b/sql/connect/server/pom.xml index 24d74b505521f..ab19897f64285 100644 --- a/sql/connect/server/pom.xml +++ b/sql/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 79846203864e1..febf7a2beddf4 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 270d23061f391..e3aa8f1f3bb4c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a1966cace77c8..8c4087372913a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 55086c6b5de8d..e51872a36ca5b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml index 94de01aede9bc..6e4a4d64f7930 100644 --- a/sql/pipelines/pom.xml +++ b/sql/pipelines/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../../pom.xml spark-pipelines_2.13 diff --git a/streaming/pom.xml b/streaming/pom.xml index ffac010cad1cc..9602aa01e7c4d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 934ff72708094..cfc6a7150d858 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.13 - 4.1.1-SNAPSHOT + 4.1.0 ../pom.xml From a96cf9985477aab27ae6a0cb7cdd0e71d60dbc02 Mon Sep 17 00:00:00 2001 From: dongjoon-hyun Date: Thu, 11 Dec 2025 14:00:55 +0000 Subject: [PATCH 292/400] Revert "Removing test jars and class files" This reverts commit 33ceb0daf9b434ac5fb32e582c7645200ac5c8b6. --- core/src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes core/src/test/resources/TestHelloV3_2.13.jar | Bin 0 -> 3827 bytes core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes data/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes data/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes dev/test-classes.txt | 7 +++++++ dev/test-jars.txt | 17 +++++++++++++++++ .../src/test/resources/TestHelloV2_2.13.jar | Bin 0 -> 4118 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../resources/artifact-tests/junitLargeJar.jar | Bin 0 -> 384581 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../artifact-tests/smallClassFileDup.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/smallJar.jar | Bin 0 -> 787 bytes sql/core/src/test/resources/SPARK-33084.jar | Bin 0 -> 6119 bytes .../test/resources/artifact-tests/Hello.class | Bin 0 -> 5671 bytes .../artifact-tests/HelloWithPackage.class | Bin 0 -> 635 bytes .../resources/artifact-tests/IntSumUdf.class | Bin 0 -> 1333 bytes .../artifact-tests/smallClassFile.class | Bin 0 -> 424 bytes .../test/resources/artifact-tests/udf_noA.jar | Bin 0 -> 5545 bytes .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../src/test/noclasspath/hive-test-udfs.jar | Bin 0 -> 35660 bytes .../src/test/resources/SPARK-21101-1.0.jar | Bin 0 -> 7439 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../test/resources/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes .../regression-test-SPARK-8489/test-2.13.jar | Bin 0 -> 19579 bytes 26 files changed, 24 insertions(+) create mode 100644 core/src/test/resources/TestHelloV2_2.13.jar create mode 100644 core/src/test/resources/TestHelloV3_2.13.jar create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100755 data/artifact-tests/junitLargeJar.jar create mode 100755 data/artifact-tests/smallJar.jar create mode 100644 sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar create mode 100644 sql/connect/client/jvm/src/test/resources/udf2.13.jar create mode 100644 sql/connect/common/src/test/resources/artifact-tests/Hello.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class create mode 100755 sql/connect/common/src/test/resources/artifact-tests/smallJar.jar create mode 100644 sql/core/src/test/resources/SPARK-33084.jar create mode 100644 sql/core/src/test/resources/artifact-tests/Hello.class create mode 100644 sql/core/src/test/resources/artifact-tests/HelloWithPackage.class create mode 100644 sql/core/src/test/resources/artifact-tests/IntSumUdf.class create mode 100755 sql/core/src/test/resources/artifact-tests/smallClassFile.class create mode 100644 sql/core/src/test/resources/artifact-tests/udf_noA.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar create mode 100644 sql/hive/src/test/resources/SPARK-21101-1.0.jar create mode 100644 sql/hive/src/test/resources/TestUDTF.jar create mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar create mode 100644 sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/core/src/test/resources/TestHelloV2_2.13.jar b/core/src/test/resources/TestHelloV2_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..6dee8fcd9c95702fb7909c909c6faf22fe93766d GIT binary patch literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestHelloV3_2.13.jar b/core/src/test/resources/TestHelloV3_2.13.jar new file mode 100644 index 0000000000000000000000000000000000000000..0c292e7d81ad741dbc11177224b9c30519b7a362 GIT binary patch literal 3827 zcmZ{n2Q-{%7sm&K(R-LdMD#j(8=^#K^kI~U-i1+;SQ0f>L=8soC4v-0?;?Wey=O!T z5jBaNf|7)f_SNv^;E9z*dsu&sx!d1V)dp~Gv2?!36X$kQ5 z_YPDV2#tzjmc0Zu;rtr#A>R@p5posW!I;j-<0NIAw89NExsqJ4a%G+S{tCys^RCy9 z7tSALAVRr9RL(cKcrVHz1^~`Wql@_G8Gz@1Z9UwfmkQ{=6r_JD?C)EhM7Vs z`)Mr`ACu))+B)J%U;>K*P5NZVE6y|h&Z&*Z@F&v+{xF#6R$p^!1B-miL%i1^?@9BE z%*B-xOkAslvPfp`I*JK}FR=!`CrgJNcaCr73OyjR4o{S+vdHI`eKIpn*z8I|MirjY zbE`Ogc7@c;dA)o6(+IeSdyi%b98O&1E{$&Kr0lo5$3Q^R6U7#j5<^LiB4vdX&I!E@ zf_`a^oQ>-YMmR=noAuqWN%=sFO*Gq+`!F@&c&za_K^&}l#02RVjmpNaV0)4}DjZBw zU>FFvRk;8}Xgl`j^j=9~>NM`FvH1Oy36=%1`q?zU*VEe}UU^gbPJK*{;4NEq+6J9n zgNilom!Uoru$T(-!n0lj7LlokOH#vKT_g_v?5l5NsSH_+%j#x&*LH? zsRGpvdMh-;^WIge89}GoJ?A?4$tPmzU5#9gUxBrTk`HUyOrs?Qaor1BFrN%TT#hjE z-hPBft&_A(rT!e&kQ2Hu4g%kln-q@df#CaRcHS#02B9I2&u?I&2!9_7=IGkB_b^z?YkE9)~f$cke}b z-S(_}Wd5F5?3J;^clx4H9z$Y7G2b7Oi?qVwR#w7#Rw{Wzh90(cA`q7aoh_C6 zq(QucxRX0TkZO)~SY)IBZtRkJ924hGf!bS$9CTy?dVL|WzC>D+it~xxefF(gl8eSp ztDPunJ@0HK{GT25-(5ieUtRFC36e})4QE#AZoIPWu6CPJZ3e8=i*TlBZOgDP*r@ZS zYiVnPD3z|gE^~X5(@`BI%;Pq*w$(*3*{#jRGOUw6=jtB}sEJuw!8OX*G10)wXaO$kkG)K>kU!uQv$Tb;>WGaB($W2(fJbr5Iher0v zPmuU>>EX^Vh8Pc1Q@A?pgt-#|mC!lXoD$#Da+9LA-UMje3rPriKP+)a5%hf-GMV;J zPs#T&S@pC~7UIp323(hla@W!KG-aERTrs7TC*%Ce1zvf&OJ$+-UnbD3fup^sm{e|L zZ?14N(^lSSgVW^k^MR9eioJ|bm8WHSb(x>0#JG;|hdy#q+~WaXS3?})Ctn%y(Yx#D z)D9(yq#i9dqcZFeQHJIs58Wm|xUN4U&BYh** zCW*j;jc-i!w4>EgtvgL$EmoK!c|SP2%CB}j=$_x_F+J>>PZsT9t)Wlml_O6ZeaBYl zDu@-kHEY%#cjrs=iB~`}Tci^((QlY1Yz*!(VCUP#_Y}WIU3Ayml#Y4!sZfSkO~P`o zxSzY?QicyFJ3R9=C9{h+ zZJT-=65*iwIdt}DeoDYV(p-U(( zkRdallFF=EdK1sc=vGK*IwX>u#s0Z!2swx8X6o4;WQL&4!!6(95C(n1I?JtBnUBwM z9=2#r@9q~6kfi5jeDnNJ(>bT1f!PzOc~`}>ytgG&;ab`t8Fkz|C*YABT5_j`I$AsR zczxL5(^?{!cKJmHZ}aT8K95{1;S@8);)sPy@_6lhs>>c~;HkW<_iHcrv+f`nR+9Kx zW22@uszkMv-@8vNZ|e+Qom;x@QrwozHo108u-IQ;MRA~PwTp4~f6%>!=~XgA|F{+G zvZ-JY|ExK|e|e-huf$?Jll`=f(I=*(FY9r#)_UC|v+Wej!n9G4U9N**OPq$ zEe>$N>Y&ztv(x`JXKnn?5{Bbmk~8rjC*KQXLTv z-ky4m)=5nj3-oCG$Tp~? zN3t+NHqrzg18lXx2r|BTJ2fwT%-YBrt2i^%t=je>D9#W!c-Aj_rg$Q%NA#<5e?p}3 z3v@q-w8L9qLs;DHUd5QpoDd(k{9#L>gZaj+BD-AlgJ_*MIkq~WJQlvvZ|QZ97$l~Q z@yZi{)=u=SKxXQL+`i~aP%wt0*~+^L4-#1;BN9NQ6moUd9&!xYo}|LcO>$VuNFp^}{y0xX=S-3`VAhu9w$qB}JT9^^`n#+kQz7jd4}4 zr$U7W#j@I{oGQS0#%wi;+@hH0+D)D@ciarXfgt@tRNjyjfz=gUb9|=8d7FIt5Z8(-*#=VZ*&jM)$kw5z4h-f<4&NHE&{k@hE-wSAg|DwXv zCokzncko94cK^F4v+Uf1W>Nxf(|$g|`|WDFwlhSrgGae{15$*aM9}RId=5tMzFjZJ z*G(Cii6}`YVz!Qvz^t>xlLzn7Bc@%q5$%6>WUV|xkX{XzEotqx4IWqi&_+5I}? zdQzRd;&)R$xbJ}o8UO?3uQxzmoWyeD+YMwbQ&-CgWoAw|Z{9Heew1q(P&^hV-AiH# z3G6RkrO@f|YjVJAM4}$i^O=oVHc0j+m?_>Co4j5z-*slYIN54gWgu+&AdRl5rzFCy zFbq2*FE zom8`sjG;S0MTnzlQz|5kLN~}Hq52~qONy*#!~F%MsugAJwsSau807;P!+l!;P>0pv zcy=ot4VCJ{u=pA@6cHd(20w~t#Zi^t-G>O7+g2xea|DL%K=qX4+-BxY%`mFR3>@XIi)_S9YwF3~pOE<~n%8@~{~s-3Vc%-FNz@lwGH zZb$6kesvF}g8I6iwSJgMw|$uHrZRNOR`Ag*Ee5F>d@LCT!h(yA(zJiDzDxBkTH!BR zNvHE%Y-~pl_4XjB8U$(Ea5ooxYkp`iMuM-XVhLuK+azUa(z1wFd*ABHnTPtCFV;w~ zol3Cf#)665kO3#abb-Wxe$T)WPpo~S>>bV>8qK>eH495qd!F?j9Rgdh9EsH9H#b_b z_CKx!RmBf&vBHipG9pLB87HouEd-i7`^0x*&w#U}1H^3Yi?*thXN2nJR0~@ef3i%f z1?HvOisoS&w2W3+&CUi%_!9$@ZW*oK;N7dRaJ=rfbTj1+=?dEm0kPy>dsHV;RvC?s z3zPl_0|(1?uf)488~8sQAZ!TJJL7Qz*zyBIcH&tu;Oad3mX^Gr0Qgw*IA`n4zK}@I zsC+_-xr1*^Y;(S|NKVpsW#)yhKUhoMk>Ab5t>zroZOp2i#I@F&k<9Iym1D=4utN^d z?b{b4)#=JG4>=J4P)>OJ2%8MEFGOnE8-9^R!$ntX{)ziW!xDXPYKR)N1Ig?#{{SQ1WZp#1w literal 0 HcmV?d00001 diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-

J{i)CW|Vyp#u)ncg$RUOm9KuH zoZhKB_DZFxFeIwTiJ}B7(6|q*WbXpCz(V78HrYxwf9nOIE0tc(6gsBFH^X9;b)Eug zoTKy`7INF!&205)A``^qC&g!<;x^TfJwPXe)JPU5{@hD4vRc z-v~;>VI)x~rZXjpUeqVdNZP*2+fJ9B&x;mJyO4UmY} zWE%X{PCm}DCp*$Q7M&-Jn@$GYc!_aO-Z+v2<80TTIc;>R;WiU4?x?eu1676@>_=}M zezs%un~4fBX#NOiFT5Y+_R!y&Qns9B>Nt#^R6e75^=yA4tX#M_!}P0De~Lm2gZBx# z@GaWPC9!`Tve7GzVZlp1`2Z2VOe&2xV@!=|s4|VA^5F6aKy%e1rIgugU5O#%oK>N0 z?){)JZ9@835ppVK+c+M}$hn$!#;VP}??7rT<`f0@{K5u(tQP9-w9KGV94D3OjvbNJ zBW%Q^A&s-}fJnr|zg0|7IqeMo@G#K=@?Vq$;kn0s4p z?v_y8(uFAPG((m`ND>)njOx*-o-|b2l7+EoR-q5_7)Ll1=ck>)_2u*j(ulZNMBE`b zaTT)kbPQ%j8CM)pG&q0GP@(Jrbpve%v&?tZZS$+l+#DXO8z!j0!mq%7oBEeDr0M0>oEEs1X$hZEY=I1GoP)BqB+{xmhpRSl?XSN$>$ ztfjs0{3&YboOOsppEg!fY?UT6B;Q!!jFEweMDx|&&6UKl6|`K-nhiVcl!?&(Wa=-} z9#IxO7>zj5=d~s=otFK>cFS`1tt>T}ywwC{ym|-7k=ofVxVl;?1G1CJL~1MM$_5qO7%gO(_7rF=?x>DH&xpo?8QzL%XR0R@=Fp zX3sC#UhRAWfxfWaxk=f&!90`Z$rZuEsY&~Cc1iinLi@2+V#Ta7p;^!zpfy52og%eg z>~UC)O}nnnDuF(^RC{2XKHR;Uk_4UxMCy3V%DsTb3oC_ruyjZ@+sK)NTQwy|eSrKq zu^1(TZmN;eL1JV({4|5BxWRwE?)}{X1K+u-+NFGnV=ET!q+E5(`jN?RFrT_)#W5gI zdQA+6$&rPFrYvP|s$*~|9$GX|{TTPY6;m??_UXqiJ_Od&SaFqR3&n*O>b)*|AsVnX z690@z92W)Sl7cx3Xyw3Aq(vd!5lHl_JL42**Ztl7mP2aF+_!zNXs6@e1cdQZDN#Jh zVzS3b-R>}C&@axm3ocZdqdz1LX-ZqTh7de5Eg;7ieX@BZY4=VRwLeNR$=|Y8E8WAf zwoa|Ytv2Y7$_x~#b@8efW88%EekmrqovFD2kG24nc$k8&zB8>-PAX1K@UP>PTd)o(n9A1A?Qan1F=t>;R zT%NTq-VC3h;(1z3qxyq#a6A-+3@^&;6RL82hs55pk+ByThyul1bXH31 z?2Olw)sW)Fb$eBif*k2k!??sexIRw8V>QSY0|?xby^u=&ebYnyjQ!S%dpyScJ$F+fT^x{$EcNIm>4>&WmCZ%Nbip6Wx`H==HRll) z8%C+mNfPKFr~}u{|KNW?>IOp~3yby`wiij8T#q|ItuSs)EUD~I~?eM^mi234*=zAgIg)!sh^`zy8S)%TI8L39U zQRnw(q4f?BbC~vLKYCSEr3RlN=NEyr7V2zg*wQbK>Ey$GoYEoAv`9~T6w=`&97|0P zdOU8oI_>NxvwrBO5MQ_ zNrPLvH?iX(+@2cJS#-09oW94bX#dMD{%}X8*02>omh65K+provqXPc$+vFnjZ49~R zk57H(`MLAwXW%u@z`32F?6?|H%SB8<)0;K3TKV-YO=<(qf+eY~Fe^?DwAyf9j}Q4T zK$b|D5_+09>Js|~E=^heNLBYtieT}@mgGfGaYWn%m)KxT>PDeHM1LIIvuQ-oAG`isGRuV*s*D@CCeS)?T%!63JQye}%9(3xE(o>Vu(+>BE<6{R5`;jPmKUQW1Ms_+x>8o&A^eDb?Q6TWtZD8 zO<|f>XYk+h+onV+IjVVyhGI{qW5sWg+V2!GKN%~BXVUz1^^knTnO;z6qjVXCd+?tV zg`T$8Y_3UE<2GuH_l7pL2gf?dJpJ-ov3!L;IBa^))5wIgn>M=UV3p>(XuAA86zSz0 zlQ8x&;6z-qMX%+I&{VhGwOFy;VOr9-udi^%|4o=Qt zZWN+7PWCN7C2m%G`~X}`_~MIvH_%`bND)y|1^qfFpt1IJ!I@Y1~^ZMvhiXJ7you;f+6XrF$p*gSV6FP*@ zQUyk@bWnGI7?Zm2+sZ7bF^&J z8&GUOtJD&JF#hB2&Cjvpl5$t64#Lw7jBc}3fl|HzQ@$j9wlcjgTe?TNDOa_MKSv|d z+X4r43G{+=6TfK=_Ep-cDrR}c^WrD#zDDPANYEqnfbKn2uai~p_R7t{$W&`AJmK@q zrp---+NG~$teDP8YtHO0G%{;jixJX@|JukQXuCU?gB?P_uY+7*BIo&%prJSm(ZI92 z9}ejm{Ua}@V-2AW^ z0DVWBcdl2P=egc5_DXu`mHx#v=Ehjx9(*&6kl(Yh|C>Je-yGS$M$bzByKN*Aa5n!R z?W2OW<>EJX zgdr(tRR?|I_;s4EoSHt2jd5EN5aBTx^07@yV|AU5ct!TEw)B)c(y)w$pql74e^YX? znRmP`pyp1j_%mf=RXRxy5AxSk!r0YSOU$3FkdEK!pxRg`)cVE@k#p7HfN8~3HZ7J? z*^M%u@2`~C9bb#To7OJL7hhD{XcM(S6ebW*hvQtGnBiz-+*AGc`v%PW_kg`&G>I#- zXF>Dy(<6jBFMH6VjN+u1h4Cc0x>;f!gL$9KU98d#vrN;Z1HqDIc|`Zr&b>q*EBvQ0 z<78mxc9tlAqz(Xz8nihQzt+WrT}-d>P-JxhwOh{9Cn6Hscl#wHdX#XVj?-+T7(Am3 zicGG7;UK+@8sxyf*1^4N^$scQ(Uc!eze1d&uqv%%S)SbE1C+pwi2RR8BJMv-AXL5) zL=*G>8a+fSXi8%7BY%>twzdyrMz^f7L@$L7`UTD)AXrI4^R z)vS@`f%x+ID&0h}0<0J05MhXU%fx}D;Os?t!<=ya4a_~pIO zM-=kF1k=ycP2!Y^5syH>YIF_kWDaYO)ov)nzbC=>KRlojLuVNh0iM~L=eg#kuLl5y^6_S zr9fy%zJUh_2;~q;u<3FdSm$Evo>A#DC#Tq1AFIHo(^b`fGJkjo_SMwB9OzYNwT|k; z{7dgAjSvFVjhWbjNu#S7f-QP;F+RFPN?0fX$8pYwKXig`?)p5Du0lqSSFFvmvlqKh zW#Za{X}Z43V@v~3=u<>7pJWQDw0FD~QEX!IrGWky=mjSRJF@t+aUW!g80yFmt^tM% z+>}~|ws(KC`@*iAQSVLLjh|nav>_J5aySJf(7!c=**TwUL!8@p^?+OssVxa70!j|C zxyVALM_Eo?N7o4s?xB04U!zjvv%FpZdiIhKNpH2^&tCg`iT{765&VC6_J0W<{;!=? zlggCpxANf^OraE@Vy==!9jfNKgjg;3x6I`P|X-c5Ac6`}O%Px5K2OF&C_k zz$!Eu-b%i=uW~U$jnbl-w-;k}Ro=QB^7bM`l2blwoQW2X5Ral?swP)gY9J1PY_@E* zZZ}M|=?tUFKLN79(iI7rzW?C}NnqGknU`zW-xxjCKi;`Jd3KM=tc}zbWixdeLw?VP zahoD#fSo=&vBt$FQ+jx$D8|Oxs|@Rsi{Q^4j?**C)_vk#}RKNXcYYWWu${ zr~ri8Y`W`6`6kLeK=s=SLAvmx;B=F8Pt7ilsz@4Cjb z;L?FdrxC`>AvzHpC@b761>r@5-BUR`r^q$L$rOV*GW1n(9}s>R#Fpw5S6l>uOG|k1 zKA+7lR3m&P%G9+(Du|Nb6Czvm4E56dGfsU^lPHGMPSV4t3D(Ce^R0`?sPZvJ?r?1) zk`M=$msSELQHIIHII0N+yzI zpo%6@lUz!=lBHnZ@YBXnGNbH=l2W$R^)CxA7VS_;nKFeW2FNpi+i4XO66g%`f5I>4 zd2&%lO2OFfVNW!Jx}r>A;`Ywl;Z~*~!QcSEpxvZ|xiV=Dv%~BI`j)~X{847{@%C9A zb%TWc{jL2LM|Sd6LDbR>@Uq#%LE8y)s*W3dHD0V#$g$?j>5$Y?{?RGXHF&OpCOVv5 zAKHK{x>a9GS{IUH>Sf+fnMB>~fF4TLvWw= z4r?$N%BG-&$~U@>fc@Chvg?e!A45_GE#dDyNlze!GeEsr=r;_Q#wSfrBEQh#9DxG|)?} z3!s5?5W#aZ{Giv-Om@zPhhYV&jA&UPap}ND(KU9B)9Hyu(QccRb<);)5jH-PZ5=1! z{0V~cQ1V zw8W&RyDnIL5-5~u{5I}Qh4=?Un|dBt7Ygp6Al%-A-pWW+DDEJu8+^^(4s=fZ8u9F^ z7=)L|sr-3d`~)J|g#|=r)s&Sk;beE=d&pYN=z%4JXJh0f!8~^0g4^)&ZBRAuaIAMA zSOlIRql6v_!4#vPTZXJ7b(k$>j6JbXnL_f<83glcfEDWM6tSU%#BhG{b_TN!xQC#I zYxIEY0Y5u?a9qL{VLUF>C-7wGP;a$n+Zbo|uU7KoU5e>nULPHO-SjaCulg$#<-FDi zOSe_{KGk4z7(*zXXnau%P`84Ix7q{QYtii|3kJW93M3{CTEYwz+P@Fdk(x+66!%ZOGxBAv5o2J= zEa8kVy<=#ccd%pe#FJIhFOf|kC7V7_K_(Ap$iTWF={2MS(Fm|+cb};;J zT)-9p5Ws?cLSi7isa6v|BvZ8OJA_|^y+{yl z;!j+ClWwK@?!pS0PA9V*ZD+V0ud}{ySUyPo@H@+bV|bbd2TDZ=qI8IuA427eU?0)t z!t(TMW9A0*I|su; zNkJ%qai-wmAfI$2*hp@t@ssv7cTozLq4Is}IN*?BtI&3Wl7dmBUax z75+J!u*wW4HNcW(ZD>8>$XlUx);@o)6?+hk1`ym^A02gA(ciSCu9XpwHs28<*2^tu>(P1gl1vvHw{Vch5J&GWDWlu-giu4m?4>}L8 zbd62aIbO0Hn`wRb&v8)CRctA zNv9#jLnHoxCjz;cc1ChBfhhk8S|y&iTSzjCGV=M)pyrQ(1B_kT;g<~ZyP~mF0kqJ@ zSS|6Wf*e`Bq;6#I1ufGP&BCTZ(>VNY2`)*mtX6T(nK0`~)bnJM$96fvt&&Pv-?4OZ z_4CD|%DD1PV{O8fUdFSX9o^sWfTzVvf<%K|DJ@~Bo)6fx5o)vsW(6oJUpizypMZar zK`s6jN#Nfl(Bpr^#PI$hx)hwh1;EVo&29b#B19`&s$vPF`;f$0)l*4E2L%6SY19xz z?7*b00+p9Cj|_GIl|5;4Y7Ia3avN7-2;n>yc*5m_V#ZANJA zC@*eKLh``kPDT|QI)Je=Ub|Ik-EGxYA+;=G|Msy|@wl&9+jT7Aj<631!npNU7u(t4 z!)e5J?IsSV9g$^@$CUv~$Y8$U6LE-YLqW3o3Vv7qWe~Le;Ru~$`d6+e;sEAjccWbb zyFQrD0W${l6|)xa!IaLOB4k%x%e3#wGLwyK#}&Z9)@l7{c8E&o*{Q~&RUhXDQV}Ek zIRG4%{{l+Pb+pp87pO$LO=#i4lcYasx}Z-8)~^v$F&dT)pP`ZZdRUL8*Z@%9lhK1I ze#Yx@2TV7Z1<)hADLzzIm<@rsUTB>XzVeOg%cjc5QD;`~k%CwC*Sfwp2^;Gc_oKo_ z4E04RQb^(9d*R|~MqGCBZsoK0TE_`~J-pE;^OyNv)lzgdAnn#ws8AV)4U<+6)E#jt zvKdw;E6f*6O{b5Lde*bIPW)+mZVcp*(@!OGFJ)k-tfh<864Pi2bJh&P8(eSmT}^h2 zsiXRH1x#8Q^Ho=EfzrC4vx&dbtMR`mk~OD4zofCpvUq68Hj9%yxIHimPj}JVd{aw~ z>0m724VZwKARju1jo!vwl)Y!>71I;$`BTn~+GIN3vA!@a;#@r9%%sWA2?e%#r+Dhc zw&v5lLe#EYqi`s?@jak0G@@}s-j9~pAJbJ%zk;67FFmfn_uJlW!+49nzM)ld&KsEF zIzf6@lm#icXjWy^yL0N~kt}{>^E4czCl+sGFbiJ-kumkjujnyDA`A0N@Ufnl_;MV6 zUCM(TLxPY^fh*O)oPpdKL#f}8qI7Uzi}b+~XC&AdSiWQndYi`tdSH zS`Fx8>6IrWiIKm?&9pzz4AKJCNVfX4oo)L2I}YLt z@#YP3USk9(kH*#*I|a-?0H2%+dx$l*Rp)sAg@(poe-F^UV?OdfqDTLn+4JwYp5 zL(6}ok8#qHK>W}l+l}is+RM$qXFTNl0zFRl)Nd8wlk!7Two1-tWj3diKIFdR95P_< zZ(fwq5&>n7ao8wfBI7oq$VHc*4cUfcjQj93vsB@kEmJZ!?gEy=} zjD(++d4V%Nq@YX7UaHhAPg+*mh^H5%?>wCf_Assfqe3^}x7r3I9&wVk)t}P*qkQp| z&touwO~^p86MZ{o^a;8T|Mq8b3eupX9Pipd0@|r;KwaD)z`qLoxYZqAfYT38A=e#X zP1W?yeNlS{5IyS^XNIV&o7s;sfkLcQM#$K}+4Nt(f3mWs=6Cz49ZCdEqzDPbFQlo-kCiYh zU-pY7Y`$2DiH}+=-hn7L zbcV;XrfZtRlT`Ne^`7>J?Oq1k){rNJ)^IXgNL=JVzZGxQ^W$-5Gmn@aIvxpKYz2gt zpbqUa4AsM~2sR1N*+BjlFUw81e}F%+f2k@ZC*Xc?glfI3Wi_PLgUf(jlbkHU-?H3`P!B==&qA3JY%qGhxJU8 zdd)bCI+jcuNjMEm&$o?FQ!bWz3n<#ysprcn+89dRzxVF!J%n>i$ElSn6RsfL)*}wu zBlnlwXBsA*(AeGuTB>1LuJ6>c%0SBYj*6swx$nZKiEo490Z88j51?sm&5YgQF`TCv z+tQ=hMLEx^8{Q_ac0Tw+sknc9%H&Y*t&!YqbkrUQQq6vV|BO`SezQLeQqF!t zxb%kph_eGcT`_BaAHyw3w^5?Z1fs2+$`ED8IQ*8RrRC#AQE}VP7BcEL+u8%VWthe6hDQ6PcrP2yr8Rlc-A3XYZ-y^I zExj;eG)|J=22<5(hd^hiHqQ--N(GE;OV;MpIEK!iuLPX)x-LEhLW?jh@usKLe;C@9 zU%N>rjFP3C@j|%|!RDdf*C|#`!7Z3}7D=Y>!PWsaVYxR}#om)!oUrU;=XgWV+G6f4 z0T70N0G&=6h5DREa={(p88krF+JX&k0I3|RDk3NlhNj)6>4drB|^B;WBF_QML^&VNm zHyjHxz$?T*oS%dp(iHnjI|NoK^lG0l+2$0CZjlzb!Z4RC9o%!{>c=c%bG zhCS|YTeGi@YUQwqmKz#7mvm=Y|LS~bcne^czVkF8_* zerk($YmQHy3y%ygBRaf4JaCfV@_mjPqrU=c^;--$`d|%XTMoyJoMgyH`JURam@9lg zenCy=-bB`jWN^`PV`&hHx`kr=T0NE&#Ea+kss!?UNfl)w|)5n=_d={+tdl*h5pC|wCwh+f4D+9$!tbmU*9hPJxofR+4#iG!izi1c0IFHKK^|2@Q8ZG z1+pS~>RlZmRvQ&sFe&pv#~yS&uSRK;}UII7Dn zzysoFpDQRp^X34h&)+9Tb>>PrNfwe+Ga}oPW*_Ko#wZT}I4?aIk>4B_S1yW;H zURKB;h60RaI{>5ZDFeea%4RE_Gv;x|GI;R4p~A8_542Nh9k_I~@ttAZpS$E?3USIj zNihHuku6FtsnG@zp;tzr!AFv7Xa=B>Azu&Bh4kr%$e74^c%Jl}DB``9TQ=!UXk4Bt zE}zAoow>@;0E5e~hZ;?P`_bF_zBxRx`+nW8U;mii;e-LRmmIQ&p>9PP0N9wbLrbJ6 z&VSJo1YV#pQS$osT%5F+wUd3{v>gEAirR}4i58{6wG`V4Pd%|t4MZ5~OiiW8wjeX$ z4=ys02S6UQ<%l^wwC+-3kc1>CK5oMW%i53p6Oj$&H8CBnN$e}eZ#7TB(T?X<9I$1* zw|lqoJIlbvqR%ns+_9ORNuX-9_E7CJEI_W@R)e4IdRozUru8rtsc$~>R64v0vVB-v zu%*O#2JwE}U`4l9jD-ElV`#&GFE!K61YP0PL&BpK?goCp&?c3a6=i$rfOVSi=(AY zJa8z{2Z;CB&16CjVJ?qF#l+HRaB(pi&!U`d zDnooa)PUkI*EJk*`3)i2*=~jFgTiZd8T%9MjTpCL9fzzcA5}Z#jg8>t=`LH7H5x@m z4D=B5=>~%7;xme6I~>9ru7z7i_s!-BkU26WAZHLW_U=>mcIt5=wS;6A>$c{rjQghf z0*S@apA>9=KocQln6~AmSC)$X&8s&_q4KK~$||)%_Z7UwjN#DB9Ushh(EHOI${06X zfkw)3(t}?iUOw7<9K^_b<%&Vj zQobdKQT8hOsZCu|*t3&1lPDcF6faVYSzTo=f9s6By`w2o&JMxR;yuuK@{(c}3-m}} zpOfq$*=cg2g#vTHE}i<^4AEOD2A%Bpja5C2Q@hIhc%o;7Hu5PHeCJM6?8`}rYj6mp#g(eIzc0;tko zQ8$*2HX~vSF2%5f8hs~;Y>?Y-5o{bNCcFviVcT<)x@OyaM$MAiq7Si?Ibk1u`&hFk zjFx7aqMdPvx!xk^Q=btrsFUwOu=Ep2rWs5!_$1p|`d(qMnS!j-jL3mR_WqP;>>WX| zbVTGN=I`j+$?6Z1cLgZ*22Pmlr)NLv#;ei7$w!ChwLKYDnVldD@;J6@=GMJnQn4)U z=+k*9_Ynw-?l-tc?PC`a>!Fxeg{zYr+{7_^wCKCXTSH!@>3qXiok-SuVyc-hvQ}mG z7>h4sh~jA|7sZH99&4GB;+!-N#AYFu=ETJ~lvjnv$DOL>5;yAwOaspWyMEnnm+Tnx zACy;Qq@|zCj7vS%jxr=4ddx6r_lww0ZR(KRVv584wvewpuq-(+j2X7;+M9iT zhYaOlTwPw7Ig5}ML(>tT0#~;{8Rj+u&|yc=VWm%My)uMTvA|y@n~?~~AQg+U*-0EP zYY&&DM3Lh*Wle3SsImWqbYSv-Hr9BBj7NX@`}(T~+>RjkJKHONgDn5|1c2lpCIJ6C z8>KIo6v8!IqLqbw=Dh+W&rc)rPMYqE9q`TqC;=7*VmsJGY60GCuorr1OWHqsBu zB5jqn!m`VtM#@Dd*|$$-M8@>?9@GH<*I~mM@3Z1P9j6G6S=I{fwtNaji{IM`2<^7_ z7-XboAyR+-^wOJ1AhYp1$2q__F&pm}9zzmSeolzb-Z&F{-Fh5D!Xp`Y7~TAec8<$4 z`j%?3|A-`RjlF~T5+P^-E{>uulr%3g*l8}|0sb{``@-mcAHuKhKfKyZN>I>xwc9~? zagTl@b-BrpcZ2bMOW(L zN0_ltkE~Z(4fZKet&ne429dC2>jaNEq=7=-dyMHTb=JU>-Ekd?;Rn_Z+r)S{46Xr)?%A`7>dl-&Fv+Y!V3@9_57_k@xcusnZ-w3(H2y(6(f zkf84t5V=Ol!ncPBw*cSAs<-&_8Z|NR+*n#4*z*!m;#I)}t)WinsX3BlC;H~RvE*sI zF;IV`zu58V^Y=ZD9=YEP-**+_{~u`>=70K6y4ii#AOAzn;NKQR{~NGU(pFsf2CUXW z0g3g<^8@xsSK#8p$b1@AK;(M~D8dSC^P;U&XBouHcVutJSarR6F>r)Tv2#A;W1frg zVS>+Q(`=shuh>nm+d6u_yuKjxpd%5S47G-rzy;84MUj^|=={$&Ho|6aOn_VBj`Tw^ z)*0%#yhAfO5H<#gt}_$tb%z`I*3*v;B-@Rlgd8i@u%xRvLduSy*Ttb4X)%ZDkCdpj z^;-4W6D!n&n+mHAsW)8~UomA!lBG?}O)ZDXuw+zzW{S~3i76%n>SK>#7=PG1TMM_O zG@>M$7_>>~F#ckm*r{VBXD1E>j&6glPzF?}Dlc0Epi=kXtLX}Oh-z~V6EiZv@S_pan~Fo zQsz!PTi?G2B14og^-wCE!oVykRi&lmt;PBwKgEQmL!12IzP(l5X`*;`hD0y0dHizF zJx0Q#YtcV-fv5-(#p0N^%FbzBpvC0M3^n;qSQIKHW>lxpboU1+5Q+CChXd0X8Dzvx zE4L&&+B2od+Cc(&aF%f2KLCJBi7pTyM10VAk|K4WG(dEeJaxp-zW{7Z$u7J=0X?NI zcLySyVU>~LPaKG>$TD1KZM{UD(Syzxum@7xa4C~&w$;jl<@t1R?<&Xjm1piWcZq(N z21k_MsYLT_wMt*ZBC7K*W0tm@(5uO8Xes4NKI!IGV+uM%CARaElyl7mj#Eis3#W2} z^g*GvJkOAZhe_uJN%6xelDTKHXn~VpDP4yo6B#+$NV-QqGYc$&N1(i4>)^h@53AAN z?0SxPE)*jElcH?d&_0ojzb_D*G(uF9lx~c&g6>b4>s;a>v?wIlorm1Oq=yd74K+N` zJ`4qFyXoVdwEONms6|+V(S@u$+L6IjORy@yYkWyO{9xYKam~%JeSjtUoJ)ee>@!MS z1uSE}&yeP-kPlLzxAo2sW?_gwq~4jobV&A@g)fz`@n$pj&2rddN3DAoG-g11W`(TEVO^~p(-=d; zgo+{6*e%n%pQpvfCOyQO!*UoXr^96HXqAK2FxwMknc_`lYz(%Fj?6VYB#+vZ-3xzZ z)X`PljiJz?D8RfXwVL=TS#XF}16S}FUnj$J*kFeWlCM0u^EBRENQbTv>(oH35*J+m zgj<75a73SZ{jfa$7W9G9`7WjH7|MH9eJ_Gq-7@ABjHd0)S8bJ05L@?=Y@6xQ?**eD z!ydk;MS16LOVK_MdeUKaE4$&_&-knKRXRGie@dWEe!%jSA*ixXBjfwp`5EPfST(A4 zqN}$!eEStH*oI=@MSb;w|xU7j|#84nSgy;`b=7=9COckcbT#12(ASxq;jYaAZ zHB1Zr$m}a;GUnBeP2iiw&yY_;WrjvPWRtd&lx$U=xjznb2WrV@3!;e_78}V*ALM}= zOkE(|hZFH$0#&XDB-+IB_PB@3?)*vA1Yk+K=8E1Q=RA3Md%UPWZ}ZWD`hyb91i?wo zo}nSR`7%CnrSNbuKKy1igItSt#WfNiC{=>B5)Kmbv%0SXTnYigWe}JXOMyzlHF9N~ z;%!;=t6Y`^T}{D5?NX-AXebzp%}A~C|6m1e^RFhe5cp`-7v(8LDMeFignP&61j`7e zGufgv<3wN~aD+a?>#Lz}3PKG?KETUi@DOrTIg$Sv#Wm39@_PKJ`26$Bt3%A3m7!31 zFq5bmH-at<3R$|y49#AA?bK+2!ZLSaGEIo!epWW^oWW6L%quL8X71+iCL5n~fn?_U z^@i~uL*wT^yx#sLg!)%>{Cm|M%@5nhhaSAGTV*h?2CHi0NAVPzjWsbcerUaQ9TiphYsHH+l>tCgpjzn$*nHhW3J;3$85)^?XS;2SiMxxARlhJ(3S?R zSOOaS@ECP3e0@I{5YoNT`?!rzFdYoqaQj8jpHWKPe*Vr7IMfHnBR?4$F?b;gk`;WR z3Yrm|Meofmri$c@EIw-AFh0>@vi?92WHNT&sAIbCJ~|b?nhn=s$53y`5^yN7!qAx~ z&)ASQ;qG`)3yYRKWCtC3WEzpQOp%VWO2(h8>g?fiocOu44vljg4imDIpLJ|xJ5}q} z-q|sfmH9m)m)TCQvOEeoLN~ay1?|e`M;9c0^1-A}F(Z7o{zL6J2v@=BZJC zfnlQ1Lfv)2wsDuh&Y?{j*9OrpAlFL|-GX2o)@rPM%r$cl=>7GULN>>-kVbI#z=fEeX=- zJ)o1OF!TTgWG=7oe{EMmDZiYwiW-L4RINOrnuS`|*UqB0yy1l00XvT$jLOf&C< zU70a!Zn_L@A>-=dkdz0xB(pGs=SJQ~-7)ie{W0|5c?9D*yO70LaP0{?`)bFLMtrU? zUMBI+2vuHh3H0d#Aa8>Bhf83DQ8j2?)HR9bZ5LUsAyT;9z`p(+?B6TNw z*Q2e9HpV;h!aMp3!TFRM+Sb6~z;oZJp(p_qL@VDg>VM1^LV5@X-q~OMsSC%=b7ky< zonvKp_H*oE4f-(GUMX0r6|LhF9zT!@;+{n+&Y*dn6<5a=f-4V>8RjxMdp{ej|3P$! z(y6HotCS2BO}wb}+EttqOJ5q!kX3Tf8Ue+yi{lplC*cRo{2M_(a)S>v{tfS6NE7#v zTtn%14}<$3bp;6jA<_T$R^eOl`rq=xMatWMx!QlyKu~F`qzh zYc0tsIjD??rY_h?4|Mk4cBF8sCFkL=HI3eK`#T!*t}wM2&V>K9c)&=73O!c^ZeL1) zp-ch4gwdW$L4rjEo@Qz5U|>=;Lf>pA@RS4{w;$eBknbID zda%OL$D)U+f-F2XCju}YNlbP#6$GXH9U>{IyV^qSx&(>M%u83S3=0TPXG1pWz6>cS zKCxa!fP<~-C60)CZ|1KglSNgEg4xWjLj*MAFx5sE<(fdGg0p=XbGGN&TzT(P-)X(s zB(;3UGL>A_`=ugWtd_{X0|rfezk(lwy-dZKm;b5cNYl~e!Go4_m&K0QDF*%_4$UQ? z`?TMiLM*Cpgvo%TxIT82Z`$E=;+7%Fuy$90i3&}XZ?xmwsh^9d4n%4{-fIpG{(v`3 zhX(w+8gj6Aq}66e8ydYeuTj=cTkm3MW&WnjpCJWW$*#di#?c1UW|t8r2TN@jQnKNY z8Zy(@Mjj&5+#i3|e97Xfx;IM;3uLXmKXlE4bFY%TZNBfz zymO5761s`t*nHf|#|@8ww>~!J{e(y1Ra9(SwsU)LMc7sk(v%%DJbQ>eGt>ipwf=!1 zrB(3Tob+1O$>$MV`|uq|fQN77#&(uW5pHf%kK=dld5?j&x!Zv&I!z(wLXFb7oi8Rc|AzOk#w z;61r2QVJ@1#^V|wlhxpFI-&czf%?en4wYDaG6&`{+QJ$GV^VkhC*)Cv@-EE zu^=D|5y-J!a$lR8ckB4r^3)$8}t|rsgacoT*1JjgPrI7HN1;f1uJHnl2 z)zP}hEJ$@EA?)KMGMQXCeC_gLd;PuS^Xn6Vmz6tSHk61*ov2D;N_e8!4QO3Ym z3qZ^j_E1O@zQ$PBSY2w>++3HWRC+Bfm(+4VxxUOADkj)bi}{ry%&0WUcVBHlx=87k zS9+{KQGxfdMo)S`=D=|xD!nzrd?0*UILY!DVLEPsN@A|~2DQwHT%pW&a3_A0oH$GF zuGF=cWh7|4+)<#oQ3^Yg^+5lYn^5S0GKg7=D`8B!dG7Kk&~dXi<`W)&sntK6)_c>_ zDmOb=Mk|=u!X^Kr$!R2f z4Wks2j}wEu9jvON0pP+m!di%%r8I7Pn#lw^H=dr@_r38Z*ZVzQ;4ZsYXH8SLGT6Zo zr5_#vW*6`>1EXiuUn9i(Mp6xpajq(|qs3&=Tq?Y3mA_%^v&O2%uxRkpck}zMHI>&w zZ4;VNM}Rg0?U*Y__2>~_b;Odb+#sIJFPc3VO~mV>h-O={!8KRjK|Sw|pI;KWqKXAdS7;u^SRPDs+GY8la`DDOy7BhoaKxeBWrC@!QNd1&Zu4K{E)cnO4@jY3oJNxjL8a87tkhWB00=b)KEJ(h$cwGyuf<~*k3%Ye)?+2`Uziv-T`R!1ZQo+ zlne-45`<6ORc9VNe@#4%Ub`u371sK$9jgi0Y(ukW+)LY#oZ!i6ZW62bxb#xcILm0bq{t)8X1<#4wBMC|6j z=$D4Gi~*D%@u<$X=EO`7)+D>41S{WYys0HR>(!cU3)ol|-~hhpx-*^+AgyJ2p(dzh zquS$C1Qn#g^S=Znu<+VeUytkrZkmdQoLU#ie~+pGf|b)jkh26BJFLlzIWi%5DFl|| z0fh{Cv707sOBVi#ezV)aBdW*~9OP-#D)=Rtd`TQ~aofv{j)>=P0X_2$*7NR$1kkp% z#3h*@YnzvdFX)3Xvqc;Ni-Lfy9v2GQCE5AS>XCx8mpr?#tQup+^igNdQ?nN6eHryoMp?*TT}4_+&KVmH<;#-XnVD z+)(sT^V||g^dd+F&2aIwoZEjM3q)XJLYf<=%SnbNy&)AgvTK7a+( zWN8hk6Tb$pYr2SRushbFe`)kKFwVp8i|by6K9dFl+C`WugznNFLhWznS4Xqo;CBsN z*99Ux07{C!f$#WM9WLPLPCE?LW_MdfaokBsGCNUHeEiA4WBWJv`-6V_M)|j^F8_+X z`=fIGe+Y^Q4#-|!^nk-?w-P*6cVH{cin*{vJapQc!L68@6_{7}j`2p&PY3v&F#t0o=$) zpdqh;zc&zO|0F7dDz9C#zw-0R*U$gY2H1bw>iGvjp|EEC)d1r<{n2y+s;;g+T3LXG zXGT=P4R&8h4M+}H8cJ~8;>=n}I27e=*dzikTo*dZ%L6(_Ti1XX+6DbFfKj3`BHv zfVcxj7>z)tOz|~oklZP41C8)A5BvQ1Tn}=_gONp7i^qk*ii=t~oJ>A9`V2*aPeozaFdtO_tfUFcI0ntMH^stg=sG z#Z5T5Z{x%yXn8R<8d9@T*P7j-rZJVob()7>Q6 znV$-&-Wf2GV&xVhg<)v|9zRLebM2hXPEH%-=iFOHZq1C?+ZxBG(MR zGk!y@0e_cS?~#x7^2mS}Y^-rMZ6!E8~Mc~$#4nepxrL|@Y z%N=yypE3mK0zTlOM5UV(fB1}k*!HB`xCdv-v-af>60cu`AALcY;GM|5QLtd)=2u|ykEOA<&=+3x&Jr*CRX9rNYqVAnPCpp zrQU=$SL+1mfM*@St_WY8uDNJ_O9)VUBAx+MgW4ttScyCehv1NMlu zZld~QUrvCIs#)fV9FDLGbmyZz3mqf1L@Ou{W3=HqM6WROh|e{S+>Ad{hybRY0aeVn zVopKqzy!MTK(kunx)~M&(nBm8Tya%CR7K5`$T1mL$gs3A9w!>MHR*wq9+e6gl+G1Q z9HJBLnhC{FEgh|Oh9HQT&&_xpp1;d1zfA$L^e33(1ySlnDB3LOHHi$a`lO(h&V3$H zqWP74ir%ct^j%G0cOQx?4(g7(Du1TlJ;BgYDg!V>WN(dkPRP%*SHdbcs)*@QpA1tU z#4&zLh15gio*Fg}lX#iF6mzsdY=P7PevBXsG~_mc%#L@h#uqHb6CT{iyhrFEALk4C zyA2=B-a~?OGd8mWA3QqX2@GP#h@!Gda|7|T27v4Y4FclBG9?WoUuBBcFZD0}R72U^ z4J7tMvr6DK6WyLk4Er9oo9w|TrL^!b(fDqqw9e};f6OF3Cr2Qsn>xDa-R9ly$hQ_#ynjUzL*sA$rXyL|1;=A7O@M+fKDhf@|+}Vz; zaXs#}W+$l_LD9Hj9n2!F`iQ>Tp89#m`&XWz$1sipE<17wjMk^ zj|(iOm)!`-%#w;G8v#2vqXL(k@CJCV-46xlEMA`2GA{jRI0;{4sAP$h%+=5sVZ^JB zHf)ls-ZE_II_V#--y~c4Wc2mH0>4@q|Nes!{EJZcAFS5btn5FMW&EGXGU#)nRtk`d z7l?XaRvk|as0d9H#IG!mprAoi3K`htyGgob%~EV<(QX==f4Ml`^*2sr{l!|mf{C5p z%{iW}@yYQorGc*nST02e(LFv?I0w;fIn+X`Q8h*MZdup-gey2K9AE<7Va{an6Ex@% z`PP9L_A7ROy;7RQAFvxuSLq$+)05^K^{sQeT}-F0jD;R?=q@K&Ta7RHepEb;g1XE? zIa_vCkD68H&}Q7|Ke92p52u+~lSN7~;V-LJlOye7&=ozR$bFH9e5Ckiw zCx?G5CXhOe78;yAND%%uo_d^=6gv3QKcE-yT`fcp3iPeYDLk3IOJFBD!l-TyLVIcY zkx=wRnJ?t6Pev2FjYfuO+P*;W_q!gRMdR^=Ec@tJxXnfFBF|1E>6oLK&X}R;ij&} zG^-%gW)J%iNdkeak8p*qa-@R}X^SF37wwKgS!?&xS%K**uv~Y^Z*Ym+oTBfV1(ZM= z(f5uk+flPm?XfQo_l}3P_z|6N_Pd{Ls|ofE3~P%l?_OQeCJE6w2K`0*;i|`g1FLF3DOAOFGC zH)jP${Z4);DULc0f6!_`dCgR%uBnp(W60OKeEsrobtr>9Id>{W&y;J)h=56F^s*J% z)hm>~RUSuT%XK8-EKxnI(?+5%oOdJ8DY~CPu74vI)-1|VbT`!^GOzSk; zg+1Z6H^hpTXY_9P4j1M}lxAi4*$j5k79pf1VDeZ6ogn+Q#(Ow!QvHk(MGd(01l*!l zv3`!@HY&|bFSk+Bj3AEbQ1Qz7pKt`+K1fp(kmcs@iddL!<@>ELn&`LONq|+kcnO8PP=ApU>KWE!nbZDq;(0>N4$l+dVytPY z>J`8nA{;$ZAYLP1WPk7DBz892;rP?bS~aZ>0MoT6tb;1OuC7uZbLgTIGE9PO(tarw z3bqn9v;RT4x1_2K$WE=(>g~odq(@hE@9+tz$Y?#RK8Lc_{*Xq%%2R)2Smdd#jL@R? zrzBe~`|2sj36(RD&h{D&kTdveBKeiVkCLX9D;iRW)TGn$rA}*+z+mU0y>Wg@Tgz&z z;HnG}F)1u0NY>wj1;7qE^UhoxVaNt9){p9Mtk}(!PG=#73McCi?inORl)D)})Xg(j z51aN^K5Cs@5kYMRNS4qINSt-CfIFwzIzIR@2Mf&+r z1lRlAs+5w8@0%r*Syfzp8eGXFcKeT`4;veu^g(|Op1O686{~nk-(;08(DF7;iO#do z#9&2;_YmK_aLm1LKatvdj^1+6^bmY)G9PhEfYbYkZ3aGZUHVfjapSu|oD&{sV9CEj z)&oJ%bqJvL}%TB4QZOnmIqhM+af; z$CeMhnPwn#?d!41phGF+2`ASC^xOPlkTMi5_V{0s%Jpxt!uS7TkbeoH{2Q4M6)so! zKV&{Yr?M)9kt-6>J6%_U=q;Bo#OuiKs@<-@p7Q%c!A|kFyT>_Qaj*U%i=v{Z@gY$pqqV7#2<|hM+ts2j5MA-cF?%!0pjWS*c)@6rFM;&j)6$J3^>{Z@X#V@q z@x<@2n!_x?I6h_5EDP!`LuRRZX$U0I!e=`p9|S*LNXtIc*N_N2H0Mf zOjjWN%l(UgdU23iKl<6%llXmk^1rV-kpIi0{U;9*C?_TJhlt)9APO?@FE;d0uspQy zQ&}O@wGbhtf#G2F+oIx0mE;g$yB>cdN>!%;rB}<{JqghulGs8|l+aLOHq+~n_ zu80#)l_|@rOe71-D5S#JngBRKH%F2+N2c9xAapt($XlLlp-Fr{pXdX?N+iH8!n*F3 z$|I2T9({)|cWpQTA^oEx+P{m|wvCNA?l=LOa*B1xOUR)LqjHn%atr1yi&XaHb!Nkt zk6Ntwf@-eT?v@pD*8l>(r_rykS zo$1du0-Sl;hPx8Mn3~6|WM;Ci^Dvpr@UXw=tl#SLd4|;e4u@t3@wjfXAOiB}wa!-` z)<~faXkW0~P15RS;Yd-lAT=9*)lfqMH`lA48N>ren3&!l^cr|BH9Q4O@!W05Cz8V! zpA_gpvB0v0dx`bfx^oQWw|T&Myr5(T~Q~AWHwQY!fAvuHW!Dx`5Bj){N8UwYwFxC%l5okRDXd>B!qda zGAO2*xtm=Ut1m}ug-n`SOKRiDE&8Iw83SHJ^T)y2FpyA8Wg-?lwjehY@QMlfAWI!A zoZolML@K)X`WS6tpd=SaD~R1bA37v!QcOgHUG<=O{Ga>U5er_(ivCPT*R{()bm)7Gd-e?NZCM zh(P7|_Vy=+J0HumoBA@3uD?yO#Q()P?f*0i2iiZJRKUjC@#}uaKd;zjs{CVbNRxz! zP#l;j&(}8vp1!cMk{Ed)gqZ~x3OW3~msc1aT8y0$p%!gz^>stLvdN-Rrpvti>tr`g z_Um%?G_HL6R4P6IBfuFD>EYSRA&1A;_7JSk`|D8lx3;T#H0l^3Nmn_5Rzwaua)SN9 zF-p{`N^gUC(@L1^B;g)&Uw9keyOITd~# z74DxV-krm+bQ}v{Z{Fl$CVyQY6hdM=z(%-)EWsKrPW*_-oq&qjO^wJKn60a#Z2Ja& zWz?9E7dd*&iNr)?{QK6P$fo32l(WzfdLe~};B*eio<>}>??h3U7dhm?@UE))?kK1Y}fGBr^$ z=${{&cBu`dOcqgfa9LR6kd@q?r@zD#3K*E^{{$L(33?F*4F;a4?h7$8ai}Btt`_NN z(1fdu)047D2+f@|fnFy**2GWt6ncS^sOSOm!#`IJU{aMuX#8vY&76UUBiEGhMlg-& zL(o}XI(js1bTMh#OtSppfdgloK6D*jyhzXNcK`EE1d_AjhzUSdMXOp!e5pMJ zaSnM4W;*ng26T|GRk<0|vUZpKm4@Ej&HW#@<;F{^xm!B=OM3(@zcp#3h~F)yjrXXGeP5o+prug!m?!I-Ieg~D<1abC0*9Udo}qP z!;RaexR)v1-AU$@m#$zMh1FdKcsoRTDkG?{YeEqX?MUh6R!vv&*-#cY+*VP^NwO;`kh$b1UDS0J=#i^ zDsu(A4rfFm;HT=pJ6%zowp_2x~C6f+#vB0z{BGy@qDN!N`^H#zr9|7MxYju`f zX#`=mFb=H6%0 zt=QDFR@1@4nRZOR!at=-%0*O4I9*OBS`HatDrJ>Zvnh2tX>`g}Vltqt>e3_dYu-`x zhG}AOCkJDCv-Vr=9yi4x>UIkdsgxFz)0UrVhP$<0L?_eYOgefAvByabf<{f^QS)i#Dhe@1q5xciXY+`>eKXS?WyDjU{e4$77J5HTyUxeXJwwV2{9aj{4lw0*C;2d@0|`UwKf_uovJLk@Fx;%{=6of7jMm=? zQR+yF<;4^OF-ho8NBrEzn^fJLOq~Z#G4P&K;dTMbUE*ir~sYGi~TSSI$jWA0SL&4%wKciQmw z!B*`)stxQoG^+`|Ace`(qFsaC9>vp4CGdPdtH)n$FHf++8K_T_=h zp^mH3P2t`58~H+$EU}JCee_$6BwFA~T)cWf55gthK(qELkUn(-nW1MSa-FYE@In}z z@{-SYZ5*UEPv@Z0y4v81O}TMaBI%M6lnIwu=wT+>Wa#{ky1rvHE1(T5Z-HQHZ8>tT z{k5V(Yy2+%a=?xEFkXme3Kq_f_%2vjZ$qi5PbVWK+;AoSv@5O8Sv#d$*a!1TzO0ld zt0xHf%pDU(8i+J$rgVDPIIFfJvlwR{L;v#;So;K~)b2bl_r?6uolig9&|=DpNVd*d zfn2qwTesOG@RxF}^5b1p6<6<#Yw>#I>TelBF})dIPiQrP_{NqxH%17a3ng2ddGOb z7!OF}n`4hf7&Om|(L?CpvCY8!CVnlK``FUW*_Vf*xcrq;{SZH?IQAI&XD_iI5x&@G? z;g6|lVoeQ{s2f!`T6TyRA|p@d>wCXM6+|J4-U=R=HYT@yEvA<~s)?86?WQ_9sgE2ALo-dowy#8Aor+j|rHkO3A@uU}(seE~bQRG-5T$&G&G%d`_` z#Ax3knx*QMR7LeJ=0@yxnly{C%0bWJ!Kvqkta>3?KD5BpyY?`#)$P}n(Q^d zwB`Ec!;+J?0c$(C5? zx1F_CT~X{NjGJgp8Tn;{AjitCsTeb019r`0Q60y<{8(85?weux6JVOhht@dEY)pk+ zm_A{X!RXdKJD*+0pL}Q18uG4tMe=^`!?IYwwe~_1C%!Lk?48xOa&S^x_x`dk@mdH{ zKWqKIgIcH6*o_GS2q87gG0O>L=H6VQN`9fcTp=uWh~P=`;7J`U_QPGhdQQk|jD#FTw3%o0S4 zY(p8oVN}F*NS6zKanO`9wdKq1^3M=|xGaM)lG)RjwbA}9g`)a@WpQHG=3i@tUvkNc zZnpnuH#I6+eW6%*@1%D7^LxmBp@1-2GINcFU~!&Cd{e)Ig4`ln_}$`sto@VwD9uaJ zI~6gpep4pT2N4XDWb;)~Xxt-hhWHgXdYBQen32GgpHR;`u1QZWo|f=Fib{ zKsUynGT$qIZ7W@XtS`(&!%Uf`cO9$YP5`|}p0jM)EI`x{NJDS1WE5%NbB&oD$oJ_ns@Z=J>NIdgC7m%K-~c}C@==uyzCkroHA-pq zH>7VAnE8km!j4d^EHZfV6Co87y8u7@?o9hsL+jiJvHGm1*eve2Q1;yw#$j&Psj$jH zG>CDIZiLiOJipehbQ?ur{{Z4^=O`?wy=4iBzMokV?@N+(83)rZ=$u`#*O6gisqKcF zCu&sl7lbqP%7+hyZELypE?(>UDnlnKCKvteDpoZB9#@YO=NAmd!9bAQLp@rJxsPGe zF-JRN6sE!RA_(7M8IjvbCw!wCk?=$%)MsS}VS5M3g&OA6HN?Z?lWz~$fjtP!ZWFf` z?(jzR@{Q9@;~6?S_UT1gKU}}&)kTFT>4M@S(NKiYNwtZ2BtbdG+F=w7$Q|ts3F?X` zb~^sJ9MKXiH_oBulld~_Y zG;STUdgs!S%fd`*`moPpiIDnUw%2j%LVaR=U}csak2eUB*(6Et8L}8u2Pm|Atbg;R)L|B z-%SpMz%omwyP7KX+9~NtogZQC7#76zLJheISu9GdHc0X6VoN+^H>{hS*F+oZHH&Tl z2I!ny`(_?fZTK1u@#QQA@Lt%(l^tUIHE6)&mS&uXCrQWPEdr+riOtO1Wd7XhfpuNVY1yTGw(+*Xx%Mob=* zgI<#x$#G113{CS>?L)fZLK3gwBt}#2F2$>6E${=G<0;Lx2o#@#XEsYPvt#c=QVWit zbY-VjX6UiKfU-7W4Cb4-DAOfP@gEhP5(%a>lmQRnMC+ z=s^>wg(e+$Wy-X8H@Uj{z5XAiu_H`<>9$^EoMBS8BJqOrDin*(2Wx|SJ=UNIYH(a0R@MW&jAz?CPy!wT^pLEgQF}`Sr`_ZQjTj4 zEZO#?22}&ap)55eg)j>v8vO;z@ego$Z&FMuJYtMZ<5Jq)=s{DkYR8xEtVZG9_7m37 zdBwjgQdvbIns~d{rNa)fCYUS%noWP19#kS{FReR|Hj;6ju<-eZI3N;5K($csz+OPM zkm>X*2uZqbyQQX28t zAODbcA~#4~3v6|NgPVir(Q2ezla17$`BBuWp#B)JkK~A|vlJ>rKDO$$$QZ<;PyRUU zrBpIJS>q{>RGr^{Q-VV=4PIJ`+y1W1}r^P?19l+<(E}l?ocPKwsdY z2u%V->Cs}Am5H7R5pLk81X4nn#K4}TRLbhb@R{;exocd^6M%QZk9I4EyO3FumCFLDe}$kD{$ zMsu|^1vzK3cp6iUAH;f!@T_t4pXzSVU?5giu7$$<>+_l_n9*C1gk`hI!d#;gUB{-c z#+MP`lp0Z29eXiORd1*HBvWHhYfA_CLov0H$XyljH>(-qQ=cP7WRs0ntLjPVHsrha zG_AUf{U)l<(KG`3hnq>wU9(jtdTF~zT3>EZt}89!v;3(qv>r-C<;ZC1qF?2=2;Jum zHWh-8@6yM4|0SqzG?w-ci-?hB1&IHu>zn=-ivRduSj3;R zM*jsj{zbf0Dp@FeH$wJiX@=3ub@a1ugn85q9Vjc*T#Zv-69*vnpD&}*OQ+Iev8yjv z_=MOqaboWaW+%)wFv;+C82k#elNTfbMA)YheOFy3Qyw%^9$5~XZ}*RvKHpj(x%}lS z0%9Y#VYwmqtkH2@a=;#4TWNRsfb~2Pdnj<~jH&&dECsK~Y4a~`QbVMa51$B!OxO^z zMW%yMK9=>(I3j3!m-Vg@Xm!F|sql2kg3G?S-Sa+6j~s=`4O^-g86%?tJF1!wjGl-W z66oqT8lp0QHRY;d%&|Z(R2afayq0B+EJO?w$}3BTnaq^>;;&-w9_GYb4m{vT7<)2qnY#PxWDtK+d5mZ4}gi^~3LM9GZ1A5!n@$BnfyyWI9f@(4z*F+@>{XHX-4Jee;{Cte#b zcVRBl%SmihgC*31)RDWk7h)b|L|mYl5vY{U1+ORQO1gq^27$h%j}~K^x;JA&r0Wef zK|)ZHZ1kqs!PZQ&1(5^jG%0MeN*iUYLMv7gJQEu>(CI}V6e{_Xmd_F;%OTUf_{CZf z@I5CFn(47Gtn&D^j9d>D0|K)ot?LLn(UIhZ3&gYo1eYh#76}9M`oN!qN&XQglWz{% zo)MWVp1SP0STN5tp@Las2}ILIz{H|_aBYSi*}}BOQfx`AG7-A?{&g+$>ZjrN`KeVT z6U5eGz(E7*O92bkM0a&A>fwIUYnUakRy$fv>CAwUJt3E-0uct4do}6MDR`;|f1RzM-geI3f}4Lr!KC-Z0f3{DyKPpjoWBf!@z4 z#vONTcU8ypSXWIN$3uwzL;52$=ktf$Kp@fkXFu-9(vjP6i#%*oxJZgF6FFF{XMp!$ zs(sA^4j=TJzwak>(Hh#VqW|}K=kWRO5=VI|M)1wxH;{H&R=%Cum*GxzS_CmFAr(fm z^k?Z#lg*hnc<+#29K|#N6f{cocyP13%67as_C~#InggPY0&tk<;!DS!`>GosfA-H` z_Hj3@zj%sfh`(dU#QzsD?SI3L{Uge$Lbxg}Iekp%45-Fl_+pSw@wB z2gJ~`N&*J{X_aN1RnU`xOBOSzDpi`e(-@)2%TR9vf~pcV~O1$7Ji_+hK;s;Q+4J2ZOH^g|gr4RpK}8 zZWW*(L0fT~HY&Rou5x`#h3`S*9Yl)1khTw9D0R0V0aLnJjh^Z}y>+he-k@N7j)z8^o^1(zuD~V98(#8 z5WugpJIamfBsu2B_3E;>vejy9_%h7r`O#u&h5Lk3Wyl!{DG(rZrE+Gc{31-iA0^6A zzgW#qio~&$@ueyY#ZQb^YNeZ4&CNTDSWL^r03h9U2=t2FcZJ!licpfG4UsmGFc_IF z3nI*<747t!*(@D$ zYJxYDyCM?Fme^HB8q3%A6D@hJ8HB1D7xz}~d>ES7-MS|AO)4rgNw^%i4oRoaOwu!E zvLTO2!WsUG!X%3Kh@W-PTNikI9MMZpX!aW^pLJW;RClEPqQG-h&<4ouey zJ_#S15F^~g4ox|Bj0&1fY@skNSY}Gfk&!Yko)8@D?Bl@CL-D7%Nnk@LCdo`v*0&~% z6*Y#|hz?udK~a}jwq3UB8(bF^96*%2B}zQYk1Ik*l_v|_G}~p^T%mk5jxT*ZOe70u zo^5VQT9KsMToI`PofvIgi1Ig{&y&97-j}aiZ7k+VLZw>AJF53w6}dzY6_v}_=dI79 zLp{7VE6vy0TZP_3@fx4@k@_Sw}std6_0Bwkj+3WugoOXZ9Py(Z)>3^cB{4Fr|BN5 z3B$(2VD9-h5l;q{#=6cpEZ3MFry- zkX_xD1f6ODfbv>M{x%My1L;y(9P2KP6JI4;Ojhgl5?jOT^r5&5zzJen9wSn60=l&E zA_8dVk(LRGokjc#M@jsmQdF(RG8SyRg{pE`&3VP*DwZ-SCX-rTEh}Z4#p`fa)gim} z>OI7)FST&as&b8a5^S3xF>E;B#?vAYp#?|+l}v55i~tuCgX@x1M4={z&?u?Y+;t_Q zgxnc{wl<{=lI%6J4_PYgybXclvvElVp;MQmG704Ha;vn}s&Z`e;-byGa;-9rq52ll zEVo!1AyC-DxX_W9OniZnP}z@B38gws{-!YwROvv15leq|M>X~fWz_}Lyb4v# zxVmsXm5?4jA&+?+sq;c>rj!#U7gO`>sux=|mDd6H?@2*!S{tP=%*?#Sw9nqTrF>d2 zrFvQ$LeZrN%w_QEXM_EKTLyt1WLCyELlD<_zuHO(8(3qFBO#57BA|JStdP{^)B7bD zSFT;DdKIEefh5_(b(%vus-{&>kogcsiKtUEAjJk7p6c6-q@;%MO1>X=uq$zW-;bM!hD?Wl^cU|=hpFc*(yrgot%J8oScx0ot)r2TqT1ZcbTS^ zl54_B*AA0B?Tq@EbGs8Z5HO?Cg;$|#cM&D=PmR$qSsCk#Qo*ent_sUv=jl=RIZ3;I zmZeu3q>>J+lsSPh*A^MVr%A6KNC(7q#E;0qaOUpqxylbIxW+L%({r|*5DK*mw_&Ov zXbIV+s(hZt{Rq!;(|2f{qAJqsVqxfM1R8eTa>%v1_ci?i|zxTmUpZ)%P(?PSScZ1vdp;S*qbKbu=REoZL)?jqlX|JTb z{5FW&3GK8AFnEwn4S6cD=%Zm86%pvzw@@A8#?Y!kOX$Z^@Y*hjBzEDsuxiQ?`uQ8hjUA$igrK4>Tl@sAG z`+x><)TV{Rk28LJS7}WB9@bbBuO#A8LHw z{xv^~3W0ScByt&~Bs?MO%BFZ0K&Aykdvy>gJc2+)hJ<17B48q0L@!ws#wp@K3N|;p zI^Qd#_Jx_$k+p7xcgG4S7=&bt#(`~^M_voH=i9Q)n&g@K+R9%h-__^zk2~iWF-V%h zs^$nyAY`uOe^pXV{Ty*ln)1|G4B;rvJ3mY!PI|jJ%t2Anu>P zM`i+NYG#%xumX}GaT@y>eno-mx+P?4h^s)f@Iv* zIB){NMgX$L-OP}Td8{k=9O7eYw;;cVFazl&HVF#+jwe!lbcTal?_ASnBo24g?|Y_q zh~wDgZT;IDu_AU?z)2sPt$@@7mU(M^RZ&(ZTSbd+qKbW5m2dKvviBOiFxYA>Li%&^ zl5_veVmLrXuUT@FMa#!SMpJV=idC8VL5(?VPtp|Mgx>vg23dBaXZnD8vl#ZGbr9S* zBh{LCEI^hmtBj+8Pq~qkp8GTXx;=!2{h>w7ty;N*tHOj7mYO01_(ZnrIf_Pk`eBW= zZ+|4@&vUd|?VaRnY-biS&gdc%D|O@-yK|Ub9Dz$Rte6yB0bh zl@EU;O}C}3t@o`RP7^b&Wx%XuUc+i=Px}CHRA+d2Ijy;=-)%g>dxu^mJ}6n@w+xWJ zLMZ8SPeg}%tZ_xjcZD%92aCEP z_K4nJ!cesDMC9>v$?>1(7~a8#{<#D{AhTP%EU|O?WcCJ4+=Y~E2g)O%*^w@HN8`;r z>V=+6A9yin5^= z$;l+U9^-LMRWj8qb?gU|vk$U2#)GIZxraPT|yE7{% zl$fII#1`V1a-DBo+Dd;J2Y!PCzsKFhUkC$RAn8@(bWiYO9M?jG=eAF61neD)`1Gd4<6P(R)7dLSo01Qj zc9Si}te7Sf_*vFhy=O;h%QCg+-^ob>%*gT%2Ew(dj?eE6B=ckVS2E-80QR1}ZeEEs z5NP$ST)!sHBP&O4NiQD3d9L2;;L7@dvrab0#hv^obhg(TEXf?yJ09uNQj0v#1S~wW`$R^w`{jy72jMF*bWJ4Vo&V}mel-RQ+p{qdl{xZ zO_v$78xmYk%*Cv;!YuqpQz`?lgyC$^MW!u5EugvWXVMV6$>MHR|BD2U4 z%V}h+@Y=e&?{~vhgJ=nNN`k=kX=qC$|~L5t*qlW%lal z9f;$P52@ZN4=Q)?p^E33U1W~%Cwa!=sR-Q(E@Z^!f9oX4o6J>w5uQzNv=?I6hG zw=A)o*RY+L* zEZa+JdZ5bbxQ1Cq6Xlvn)*zbV+m&zapSd_&_~Hg1=OLTGmaA4|uRs4F{8m}map&b zO>vPPkr;YX8B({=^v_RdjvSL=cPP3lO-I<6eR>{imfk>zK83#HrU{{?l~~V{TWV+T9zcmRIv|oBsaRZ zv$C_+Ti68n8B;8@HeY?~{hREGQtIxYM^Ip=M{tX55PysM(~d62-Fa#0ur^owdHa(l zmF*lMgnNAoD(|30QJ{v&jg=3R12KbiK*xG_$-^umae1b$=clbsKEVAZCQ z1@Xj#4Y6(n(kwHS{#m(+dIlSy@>ylFv!={~8V=&Cq%)C4VflM~uKb8P1lrYyUk7PZ zj*tE-&UP+KKg2)NfSAI@Bv-~v6#f-&`$P`Bl$oYmSs!m{qeF>&dAid^hIw&s4{cn6 zT6!PO@n%efrIT1AY7bSK86O+~=ePGq3g|Xwrhk}~3hywdn{<_{uvdoJP?&tylt2Bs zW(yDMnM`<2QuhQ@_DD-2+0|*iP0e#u3_S*JFz8#fj)rV*!ReNPfyjFW?8B%JDDR1} zi-yC}6SxA11vx*vPcGjB%EmfIv;ew?8&Jm%&K`=88PNicLUl!5T`}F9eqS|4+9Tr? zbSFsLL{>(~Nav*7<2T`!W(mJ86mY2w0vxxd7QlX9Av~$Aseas>(1jO{U<(J13MzWT zvxwPFv9CkPso*q|6maIDT~7pN;E=A}%#I#8+`HRQAv-*VuQ2Py$1t%{6f?Vo6RA%f zQR8tw0miziTpae@-F=I{!2%plbBgfDtafxUk=CSGY|8~>NuwL{bggP_?;!G%;y_tP zg?)bZhkc|O0EyByho6_3Sp9@?Xf%jPBr}`IBXO@p$MiG3TRgW4?_93=XGh(XQkZg^ z>I(?PpttBoHlcxa!pRDkg$nVBYCsXz9+wCyG5*k z=`IvpZ3(fC=eM@6W%Had^dx5e;k;sjyP5+{^IBVP`yQVsjJsdeYzyTq5Vt%co4p*T z6MU>e*lAR}`xKfp(r71s1U4Y{BjZs@z|W_Dqbl@j-0hRUT+SM z_(jU8U>Pa~`HOS;=;=C&54<}9Xxoj~9 zx3D@St&&rU73A}@C@tz5c^eiwC-Fm8#pix$jE1eyk9vs(|2T`&AorLtvwUN3Jq4*v z?CZeO)SbtVJh+pJBpBjlePX&e56+|m&TlV;2vm=|xu8m-?Y05-*P%8jDf0pCiEJKN ze~?x?D--QwB`r1Iy?c|R$bi$eK1C^4(ad0x=t3PtBYUHRqM`n zyqM0+bCR6!AX99Kb|GcB?d=N$&`(d9VTwBi=+tjVf;r7iB-EP`_OREF;l%QXHF5oF zn00a{c7B@iH%EWg#wI7xVuz7Ear03osz{>Z>86zj3dxNMRZM^b1quDrC& z_vjSxzj#|SSqwbcDA(wB<{CSu;shO=nC(AXI; z+1w?A7Z%GNwr(x&wk`ZI(^ZX(XZJM?nTIRuAZwEP@U| ztR8EeecN;RO*lh{VQvD#H*7*lEXtw0p6QR@B!SfERhSS#sRHIB)%vdyKI!%b#ef~- z+0rxe;BIkHh%LcfRuAOV3!m9tA^t&D=S2@V!bdHz-bG;v5Q|x+b_67()-Yvd1+LKM6$D24e?G`F5;NI;0+E(cO0SRXqIf?PLwtOxZ*!1`qbIenAs( zUZMJO#zHw-NdfQGeF_m&pVPagKoPmG^##w~fC|V-9C`PhLHP4f0xC{FF{o zVg0bYMD6EwAbxTT;zueWwxry#KCuV5zbQvNG1o`^k%dz}Ik-}${n=j`8oFqY6f=yO ze(?XMJDsk;)$FRc_zmp#-0wK$kC9|Dzuxq>a=`gT{utg0gnFZ#A!YxV-m2!9)n)!f zy9{8}lkf!bU3}i@7Xd-^O#L8fZ_mAjGo8(C&1x>dRtMH-;W@ZvRgEjKa^$pHTt}BB z!mR7n10}>PY0@^;Vp7N@FKb5)t1G>O|71w2_sz$AJ7{jPowU+UIgSw>U23H@kFPZ} z6>w;EqK8$%0J(Z=pWU=h(M;ul3w3>aNH}mrfNO}dorwYDhF(p=&daJ&k6rb0dtkr`M5HVh6Y~=y#X0* zfaF!%SzljvdK|RbS~E2F@D^(XzXe;X5+I42ThPCkaza@eYyj-OuqpVwD7(?~jTCZL zjMAZ;PN@`gas?;>0M}RDB`p%@Q?9~vts`N8U4^S$NwT(71Z0!**~(N!B2dM&~N)b!?*M=Rxg^P!e;s zVXO&wI|{1&D7%5@_x9o6VB@XvCltJvi<&nF%ZE3#hP&e6$_o+dvU>#^RU zrCDluk9ZcyzP{^xPFQ+m+@48ri;f8O)YdqP3wI`>y}(z}uVQ`(^3a2{mPfkNxfV%P zo>6SpChBd$)6&?pkuM&V_?JkKGji3!qUC$F(iPAL4%G=x!`jGG5kl&Br%&n21_yQ1 zL|4C%Ea<8PSw%aU7OE~4qPc`^{0DglDMQ?}-F=p{1f31G%X*;ygwwMyRA@7mStYk6 zG<~pkhTtEpt;C*Sk$mjMJLQ5{fb0wDuXu4j#_ZJ!=2YJU8=P)eF&%WUi z&9MIi(~b~-LD=T&6UU2rvE`xU5f|apogs9OUt|;&{k5tj_;Jgd?83_cl*D35n=OY@ zmTfj0sw$UR9X;7`Xyf6e$1G>DPbIz-$4<4O}I3xUQucb-;S~B!l0PNcE&!NaT9Wl3ZQRO7*Hv@ zgrE;pWEC}Tw}ziv?XY>o2k@N>s+bf~jTB5B(m+~*=r;AfK%%#d#xMw zfFAIi(R+6i9DusC@O48L6^sH_y+VhUExVbqPl7r}3RSG!1pU)_$%{NVp|vGvtB5Vs zsaVT#kuzYXr17jWe9J6dBrOmAa}2y@Dej;hrOYCG;M8BXh4yje@bg6X*u z>(;S~;Koa`F7k%$u_Z2xi(~C1tZKWm3#XldC<-M|^lEu_9;EX*fb%`rnv@)e*i^eK{|{bmAVFu04b^3{zZh4&spECaG-j}*+jFu z07IKskx1L#Ex(nm#wo%ArO25n`jl7fCMaF)xpW?zhOEkV>OXvkte@t+h`sY3MOR-` ze!T}}UEsd10%he{lfyIrCwLt-0?GypD`qX(B=6vj+&f}7Jko;oPxBM5`t!3;kn#6U` z2BduFVLP7)cuwGse~Q_hGtf_g+&mTHWbLUE&`W=YD0oPKRpcZoGkntB&x9J>a}5BV)k5dM#F70#U_9P5F@stH2OjEyxSSpL+&)_^E$xJe|AH zJ3r>vww3)js@XMkq~i`qrjrEk=gq^UW5UvHQgb14E_={Jp2CEOL8wFC{L#6?;T_@= z>BuZB#gkCWkSLr2F+#A1NQAirSnX9pj#N&Ibi)Pafr249l|22krpY=BrkyYO`eP|h zx2X~_^(FYZc{|44CiXp~)VVWL7l_4gy|V#`Piz$zD!F|r)xo3L-Z9o#8eDWu>p2ZD zKVC6~(xDFRNa}^_9N;W-+m4Oj;0QN}bQdX-^}HmTceIe`wNv1=rovRT`oAW?-WC(! zx}<~*;-P)N8E572%|+bDPH-Y~aqh&QEx}vN1@(>s{=r+=Ol{?Niq^WNRM=A<;mtr3$Zq(WjsLHZn>Ipuei6%VgO*?=4Gufl&ehW z`trKrl$Mpgi|R7GcG@C65^)Mtw=NK$uW;dx593!KZC6776yMP&sQL+V<~t!Yypw@q&aS2akog zQH~qrxvN1{V=1pH91!N$mIuqn{<>KgfFbF4q2Wsoh#UkMQD+ygb6cs{qQuF9yBAj% zE~svZ(x_SE=R{!tWuL|>z1-VO0kwG zA*>mdsi8HXru$=W)HB<*CVjtdbzORg*zXm^0Xxt2QKstkYRfB|eFbW=nD$kJ%g(yi zOq4r%v?;Xy@AeN(a?TJG34c?f>q{*8@X_rnDDvzGV%(OZKFM)T=vHxfF4~1^V9*U* z7e(7NPpeDnJu)O4(h0knT(^eGy6@P9>ZSs|)xE!LBO-s~+adI0*shD1m(%Ktit*Zd zJrhQJeAEC>46m| zxq-(`6;LlNFJM*Z0i);`vYgY3!tBo|L*EqMeR@)#FwCNUQ_G#y&e47oPaW%&!5nBu zUA(Dl_qGJa8x#zEnsS3aOfdF!{OH{ektq$W}j~Y4RvGZtLCNV z{jQ0Rt`9_UUlcq#h+3zg+fCq;YMOh#doyV&f$r^=a%wkDs zNIvydtEn0zciH~r22JLKP3C|?8stfRGOOLw#I04jkoQ2Syn>oMOTgUbQuh1jZ+&v> zjoHsGeR8zt>=b3b(Q%LeRHj zDI6PiEsVZsg}V~R(Y8d!c6uje4ez@Hn?Jmvt#W9`{}|**kwT>^l!3H-`6}vw7)?&! zw!)XFqMwLZG980a|7M+(P8KjB+kX(HfS=1@hgL{kqSR=O9S=M0+A`B;gkJV~tB=|s zo{|aGW=ghzkeipbs9|4@H<)PzJ%6lP_s_xOE!6?|pHIsr_6#?Sb_@{LihVwSHu0MB zTk+17@{SYg0mmj3#~W{43Pp1|&*%lP!2|A<1$83^ZOeT#wDLlll;_z{$Ja%vSICtB zZZLH7C9}#+Q|qjd);bNE8wu)CTY)U9w^YkmUI8M+c`f4CAaT9A=~T_>F<$dp7io4n zULxo0ghK+gdbu9S6!Wr^yx=|Myi@&`WeqjTYRezY2|Tv)*VGexH}jRWC==zgb7#B* zHAk#~or6j)NiixorKg7GxXpE9Nc7m~p+dX5Y)xMB6vt3rl%-RV;C$nZSZfhvN!1QRA>vE6R7a@Ut&>m9}1E$WFF#=A$TQ&m61DQxfWF zk371S`EbXI%F$2z;w5&iE9T0jspJ26*7W3)!bW+gGN`vP$ z^O^0R_my?)+OB8*_j7?gch@ISVG7evq?d)j)~n^e0hmOJk0@V7Oi0p zEVKMxiVyTJlhwZ}&JNh<(wx~i=A);B#5GfgJ7!~X;f-wrxOO~Z{YKY{{j1JxruZ1e zx|1gbQIRTb4oun_seF znv-ID7;tN?As_U~ceJ`G(G#Y+fzik{hXk3VHpk|_5O{fT1WtO@R;g9$GptV}SBD4I z+hQ}5D-!EO!pxe5S!&`PszWYJG(#`iBq%U&T2J@)b?K)M3Nj<3k?j%;>HKmGrc-Z0 zu(%yrdUPoZAzL_SBl(ysT9YI7p+8JkL-?Uv#vJ$sS|;kS@R)Uj{P34%!~*_8Iy(R{ z>=Tn3YW73@FI9J?F})5-6BKx^s=`lMNZ%@-VfoX1sWvWI{E2S?WJOU z(je=T4cNIzdaxPKK43%lM8tK_pKD|W`SqMfgNZ;%TNCDmWqjs1Xu^>I3GZdV(QIgE zvU32~Y7BbXD(kBOZS{@DctMqPnCI1N=y0#ZY&^Vv2C?CYR@_StqOdm+Y;;1SHyR|q zeK}@qmvCSQ*WJ>5L43XpPphH+p$3iIQz|17d%ExFAAMY~qLJVEfrTS$tX zDj1I{qf{#<0iahNA-z@or`CuRdfHhq*SG0cX{F9Jjfq=Lu*T?Z*6Zx_B0HQP+Tm>J zGBde1v!ko8(dT{J#RqSoAD3O8Lsby=Gv@cHjvtph@`%N6I zL!;5PhTJi-Vx#+s^EY`x>O!SB7;UwSlcz=-6?W5Gzs+`I0`>v|hDrG33>eb22ID!u-z7&ePRY>0 z>-*403pA}IdV4xZ7>8qVmK5ul7OmeLrE@8=I0+Yq^jZ1XtCIch!8$9LAe|by5tYZr z);O{ckB|&{){lODNd621(E-PHSE6RH{)r``BOj`pc`o-0%fhv048e@hE-`pkMqOFw zWK7KjbFog86ESGj znz0$qQ?1%EPcL|@g{1JOfZlZ-*;?C48Cc!yb+{TS+1NNmqL#BYd4Z^6Lp8Pq!7v3p z#jku$u@z&_K*lt$~h4;p9R{Kc1z z-q0vSmq3`3|1^`<2uQ!`{=HzeWSZY4LwUhA9~-wRT)n523@ZTVp-T>GN;SLGj??}) zm~>=uTOwX8ZrTTX%ecJrT5@P@asy^^)Ml5zNT>2HU#TVVp+q1|KPkEmWy9y+wZwPG zVlww-#J->Jo>4M1v&!X;ZliL5^HB@D*!AL$k#(5YJ%WCjvrP0d4g)a=tG#XIgW-ij zpmGr4CERm`(U0X&L{4osSYcj|hVg^5FVG|8mz5bu>%_8;zf}jfAK0K0j4}5Gzv<;6 z`)m*LcBlmz*+m1t>F+@6gzU-R<+#%a#}AgEq-x+!3VPXVLG{4RQQSquP;k6s#h7ut z1HoW$as=0<+T#cBiGGoJ4G8+U69Oj)GN%fcnUqo&r!qZh%~)A0MgK^%s!s*btKS;> zi1ta`F^vyL`0g1w4er-T$}6bcI@;0yDa4YAt$qv%3cf7|A(1`Cq8HbCC0-VmKzIQ! z{Lb0upMeKxZXMl;e|AI$0|Pt+i$;dFJ>JWAfqj(wgJJOdOA#~XA=}dCH>}o)3p=wj ziO6u39WZmrhxWbIV} z1-rjI(gcw)ih~&TWng?D_4)C8DpeGtC?%1?XgRxqs1R>o;RFvpaca0W4jvvRJFEWj z<4ZsU0tl#YRp_^m_I>I7q`>iqj8PTmL6_#mZ0T5DX<$ClzQrH1wjJoKbZ}2PU_SA_ ziS{aCy2EN^UVXuLdO65G1A_zxTTnldzA+7jylzeS`BaP!a|suV_~=tWvYCY{@^?30 zRlxZJ1*mQ%sBiH;(?f2kU!6VF16w?nBZjxlk<6k1ja0Ar)daYM1`huy*aV8W0|xuG zh|`?)(P_4~`D8^FPJYonO1C8Re!n15Kk=@~6dH%KmARAdL(v3KJs<^2M4U2^^aDF~ z?Z7lDIwZdcZQ7!X;8>2>O#(3uY(uyX>=mGrjq4R0!v&>$)_Isi6RW%9I3uw+*vQ5p zu_nYPajY~d2BtAn1I`FAp*0KWZ7a-agdHv)R6l!J_!kZ)Wu2(FtXQ)%(Mwuc*#LvF znrwGUN%)i5ukz{l$c`}7I6Uc1Ic~;n)qc{MH`pJn7Try>>2-+^T^Xe3wcs_XS|_BD zmVYYqYR?(lV5oy|dC!;{ zTo1#Kh!1C#v|TsYP%4uzV(&h*(ihk~TBoGezX)Qyy|BOhrnBU_o-Amhq@?`v88*}I zg&8JT81o<~xm7z7D5&4IC1>XTgo}nP&=*u`=na1%Ct`<>o z&M@xMy@Gl7+VrWnY+e2|Sy@jzH`2Th)p>^`?x50*PNcQ)X1mF~qV_*#;@Tp;qZ@z3 zyN+r`0scpDI}^n`^RT6MErDjgqRX!2&X_840y&>vC8M`Ju935>*^ot+*NU9*QeRXU zq^*mqjnLMbQgk7l$A*7@I_;s({-WrTX)~p8<&Ywsk+o)!eG0vKd`!HG5&VR)m${50 z!z!&lO5vlht(*6tOv`pMKJ?3CnU!DMUSD2r?=CJM*y_wtURxwEH~xB~KO}uU7i(o8P%i5{Ra+Mzflpw|7EuLV_7k+HUq5o^eQErt31Ktr3rkji|Mr^W#&=NgoyV0ga5$ zD~FQ!;_a)ih26)uV@v8ScEAEM37W*ZWrVb}6G(u(W2=9AFT}6PJ#VOAcRR~!69iAKqdD6RbTCIo!#?dzgRN(e& zQ@XcBNiV;2*@N#%8J0I64XiQj`KG3EWvRg=5GOVv0q#ODLMn$#P_m-}IB1m}^a);d4#^_#TpPg7qoJaRn9$SAg@^fZ2`s0 zm?0X>?_oqvlzcm8p(9cuK6r&wMU69bh9fP8eIs%l6LgHmwzPpEs!tI|AFRn@EeIz| z9?w)+JKppx4{FcsL|$9?w{_n8z*pF0>}IM^A^*EgNgdIH3 zr^o~IV5yaIl*yO~%u*V#XVM8SWMi8QB?$l@S}lUas-4_CS{(ncSyR{Z zMxq7MNyNlB$}lGNCkmGo~;S1t+v=Cc{MuQT)+!&3*1*$r&> zr#wb!`eK;eh4Q8Ne4q-oPi~-cDaAI8;G7t2m1Lm@!BgI&nJS3caG!0=$cC4{! z)QpvtU2o#oDmGN@N}42@v&oBF-vkwqU!ux`W79ZI?QCQkXNEHPp@6Mq)TCuH>`7Z` z#z}q@nGzGconeDo7ghN#54{d*F(bw^s7b`OO~xwwI-V;DB70Qsj>_z3!&SlhQZGc> zOfa|LARI$#JLHB`OU*EwRHBXIiQ{!0VGeM!zS!LnQh(~N50vr1t1ts=I(wDKXn}?^ z=$9q#mHP4MaQ#Qzd%Wv^w*zyL63~0A0x$Fiia3zT3sWr5{#n9w;Mhy$V(XNj{L)xq zGDl0;GX`fy;Hc3Bu1SNh*&S(ET9dIh$0@)(C2I)Zfvo5t7CNbhI8t+irlw_UtLu5#~tcUfg2wA(! z1ZNX$Bjr8lh@`MsgQ*_tP64WUe|5nnsCWUtKlhrHZc#-CFa$fvu30)ui`3cvnA>w+ z{SD3@-sQ)p<+j`TSqmq1hx#;$46~^|pY+?dtVA^@j6wMd7WIMAm6s6C>$wAr*|0!&Zr2~W%3nSRhwTSf*8Z|w|VrMVvB&Viy%bh z&QL|Pf;mkc+E23TxG#5aN75}SdHMckvEa(wpJ-5F-e|?#bcF-GBY5x{vEDmk6)qdG zZsd@84r5l%#1C+_InZcij0mVj9f0)gtUMz4r%C$df`7c#y7D-yM{dO=i7z7~+ zO06MQ2AohHy_glnb~J*+8j=)D)@gs)@0tcu18$fv5d&oQA{Se*1XnaZD5U8X{omB0(D)r05Fx~o z9_#gDEiJ8M5lkY9r`fqiBvDMLFT}2(TT8&6HrLC(ox-ibMZQ3u0Ch$OWHjfQq1#Ev z!&-XwTJ24&0uV--&MKhJKA&7MOr4D3Q)r6G>^o(z=nt7*l{G#keKJJrKReH8)In#hMhiyL2^i-rM!lY zI9gwQ>mSMc5)4wQrp~45)1KlbmgHeyr+;FRqc{w~j^Yj^v?>g~0ee}W99Og{v_${? z-$>`SdMQeweSLb77ps!9i0X1NbJx$ANA zMP80xWi(m{4bF&!^f=v(4<3O(hdok3S}KD2fR(ulUld68Kq-AA_9xTa-@-FXlpuY2D8!9@^WPGg#J@Z@=2upfVc^Uy z@6_iB@%dbQ$Ccsg-IC@WT0b)s0n~IMA}DkN_q+qJf*GS!taD;GtP(#kyb;<(3&2~p z^Amo?V8`~cj3+lyzu$I#`xAA9k|mI(g*Sv}x*Q)znZDb_ck>yxfj;D18$EHftG)92 zOYVyFvS-r`Q;I=7sqxOE6d9j z)o%6cT}wI6JDI*lAYY%PH{0$zT_-sYxz69-*FN8;Q#aetzmcjL1Sz_2_DA^cTUSD1 zHcR*XFfLm)y7>qP1md<4ieE_J1l;y0JQN28=xK*Y=u!8S-agZK@fij|c!^?j$1iio z7IUp%LuGv%5KTR#!*WwbSnjX1;AY0u6~^}s(2dCbj`|;70bx6%Ss1Lk)q5=s?_zKa zmL6IIH@anmH*B56SH7XQyGz2K!2U@ixh5w99kbM~XFmDu0Z%HQO%J=U+` zVgBuPxG!9H43`=`}z0o(sZ%M&pSRT6kBcNxg zo|0~bvXn^{O3^p5_^_rkdQPo3r#NqP>?|zLuXh=$AQ*~yQ*LVbE76|DE{7ZUx3KQT`HOgb6xT?yXbpDWaggXC z<|yn^H1Ov++zVRGnBFF_yhC{J>R44@{}2u=;k>T(IkJ#4p>Kyl|fR#dE%-rryr zCpr{PSZw%&{DE&5?s>_6)hm84vc~3^ED>265N6az8MyqAS3VJzEzWs)WBRX1Zr6twN*H%ots>E?Wl(6CFHs^PfG9x>~gg>FCk{q>@w`>)CnH+ zhokCv3$D_h#VND@D{908QShzk&?taWUpEU$wxXTIVtQM9I#G|r~0qAI#OW^UEA?w-MYimLh38#y>k!<+0n z2eOGH`cofnCY}ZFA=MW8<#VXf(ohd_955udcM=~wB6HJ<9^o8>6DnV>LRN$>Vguj5 zcgCOA4w?At?y~1ME8#o}6Xp`WwKg1@--5e_?fRokIBlWa?^*+5-M)#ObT7={|jq~LwQdpz0Dadt|uVj;1A10!LO07@=)*k?~f=}?o5&)t1|`T*@R z4Ay)9{sGOqyUNP@I59!*$b_wt2^PQkHxpQ;ID+nbk*`Z{AZLSv@gXivq0)IJt1=pL*?hgA699k1X`<;{VS=Zy3koIDrInp*m3Mm=FmH$aSWr5p5QInD7lgb z_18xfdr!MTQFSc}xXa_N4<3@JYv-?35<5vtjJj8^O&2A zf;rXf5~2zvECW7;*5#EmQtouNQYR(j#+Y9Y`SJ^vVe%}NDKQ8KB-~RaDo{CpF}Ate zEMsCsh&>VL=k6?h#7o;)V`n|~0aY<@t7R(SQ!5sM^1sRxM4rK@G)D;S4s`6zw9iz< z@VnoNM=)~z#sZfW342-8w)8V_?V!WEyiFPcw5IM7rZTvf6syHuxtD%*!>N`p5NlV> zA!nz|mveUq+Ml~%(;Xx@1@RUyI3WRo%m$OC%xSy-nl_A`W2$s4cClp>y$IS^_ue>5 z91`z4Q)n&QDrfSby3q7HUo1+}xLPp>AGzQe^l3mx#^wgQSe2-Sq}yxQoq7oH_rhG; zt{Sws&wYyloqK z>huJWudW*BZ-@G#a687ksbv>l%@Al%cllRL;}_84Xl``!ReOFL8?q8rBfV`WVuShdmfdikOs?e zjh(gB6zB$*d~qi_iP4SWo8CNolVlHHm#NZKjJ-TRM|e9^y82~pByLV>a&{u2oV_I* z>m^>bJ@$rme#nwljudLtGO2vYbzQ4?uB|DXHYR-o>()X+b1;-UJ4MwNlJEBis9n)=RIZ8083%6tWG=!L0<` z8+fKX>rg9tF0|ytR;FDu*owc?|HYd%a-(Zr;YdH-nmt*lYfnsYv)ACuefdd%_yjv} zj1}5d>V+9YGRvDYFe|>x8w7Hm#Q%d9)w|?~*ob3GGRq^bw21P<7$SAlY$$vExQHuM zU-}GH@+V-wkdg>DAjLlp@|Ew0Qu={0sN5YNcxzaDS{Thradf>i6~GB6R<$rxr-V6# z+6Nn9h<+uEb|vz!*MfJ|XVea6KJVAGP|O)lFA_o@p1MMB%5$z5Q+?Y5x=l14V%+qSYas|4u1?jO>AVG?2z0gcU5k>7A^isn!mH?TeUrCe8j# zZ5mZRk312g^@%?~uTEAo&n#2C_4#k{Cm>dg;FLLY<>M$rWd?7!%rt3J>j$)cmBu$} zJqeyEs7#a*8cmV7;cdk{?xLE@r0~(ds!(%UPd_gqnP;LoL@YwHhv? zg(jq^k&~pAs@2NX>PtX>LV;oYskZo_Q?IT)0X%D=f~3x;vTtUGjFf64!qG3pLAE(A zgFH<;gl0ogz8%h4_n>+stveLK&FN#*nt|jrwluKr7A*{$cHM^a0G~JN4Eyl9BZRts zEhwhYT}so|A!0U9XJx*nTWk`k?=vG!8hr_VSy|$P&J^;7$L4O_2h3?5W7tRdf`Q+A zsHFOQ`_XTxIu>^VlrlEUSi$4D>YPM^9o_s7P`fA{lSSLbjo+}6vJqRrkDvqJ{UlsX zYg$dQl%^e|E7bNCRwrNg2wX$1KYYekBe*FQ_ag&+Fx+4tgi!*V?vnm_BD&10; z3UA1RH!P2>%+B73m#GvGTOaM#w2$h8yTYhH!9^v|XF!_qtOZ8GmycN@CLfAfJ1WDu zljV;;Gb5W}?a98QRt?n^+#}`W=BXM%Yj={SOUz^$xLc!yi90Yqy-t+n1;ou!iWku3 zcDt)H6Jw+`Lz%)7LL-eGUN_GYLT(r4@XnZ#7?5_y3q!FL*ychJ?0_L(?EP<%wCzV9 z;mC)RFr(Wlh(#;3XqtZZkDZicG~4QUJPK!dmP*m0OoLnEbaBYAm2;k9SY_3uWoOqE zDYMooF3~o?>`?QV6l%_Db#!TUMX7y{`0}E1^ei+T1-8|YNd~o209Q$OVJ33G=Rdj3 zH+t>&r`j8{`Vn*e)Um3NA}JZ8K?Y}a9Q2iTCTBW}T%B8U!ptYew$TtqekgkLU(I8q z6^(U@ zQKU^S$th|-N_4&Y#D~WHw(yXa<)Ok zqd%gST1qFwpy-ZQc-|7SqjZUB!U( z&FhnA#uiGqZ-oB}iYhfH&~M;?fKL9O1M&Y^G4+4^q5MxAk*)D&uqclD15axLt#qhb zwp>iyfJ;xUFIZfbr&$AI)TeLAA!s6Hk)T$eTK(8-?|jW5!?LL`rkwb$@&h;pO1Q zfI`fsw+TnV(JM!0vg3}GfcIdyG3%eflP~9l+yL>mARra2b5iiN2wzSm3RGx~6)n;5 zX%3MfbE{!z9?t0ncSCsmi-1Q_#yyb z5i`O|P3{a~DvD;fju*cuJwbnq0aC(5Uj*y;CzQ= z`EJoRY99|a8dB{iPR#X&U~Wke_T(DYQhFpYBGA6bX1JLttpRU&AdW={ax9ammqqnj z$be&ovl?ds9;QFNaCvZi=6PNO!W4#wK|pK4WrDvBwqF+Yb-F2wopf^0)hC;28D8pK zf88F7e1&~p=1hIl4B^a)>o6WY{4s7%qh0O8OplayKEw{Vpe@V@{?RYh=|D^Bxy+Jaot0V)U44FGP|vf++0WhWgH!bQctwV4qoJ z!D;Pm?s6SpU3Owx1VexUnZJJogg^e4ia+s|t2@pC+yLfPx__mWb`Qr#=HvvOg?^9c zznoH^l)DunxwnlW2DjF%^lmjq?j&2GcWXowih!{%A~wujIB|zdPtuTj!aZzsPl;Qv zNh&^6SdWrJm59={LY(Ga*D$r5RWc=WFEU!g`0y_d=$-?J>~q>cQpz)A5u&An`||0O z5L;O9Mslk+{q+nv_qTr8es(OS}Gfv&~o9?PTNBN@57n%~S#_G0oL;tM^ z9jW(h{l-ncET1!RJo}Rv_o_`Mz3CxKcO*2*?BY0P#=Z~rSPo;6@#@A74QBFr)hT-F zP%fR{QT#*XXf$r*SnMlwI}k=jwpbI)VxvsHA^(fCw+xDOO}a(9ad&rjcc*c8*T%JQ z*T&u5-Q67;cZbHE#@!t*z4y$Vb8gJp@!hyVR0Q>}s^0Zvu3VWT zlvejd*W}fN?Hg)daBJ3CCs&M))OH1lXBtp_Nw$|nW)!$&llVFk0Prq3D!+?-+**_y z@qV3QDwEDPY`aV53OOcI3S@!0gK(!Qr?Eegn4NO3`m(DOovU5reV1}$Y@Y%RX7j@ z6{4`wxoPjIu022Ko||jHJz~REu_B4^IfcWIv6H$*v6q1C(PBIji|l7aZDQ$}@m1?h z8s=Jh5r;CDOw*|a!Ow}}GyA24GgIICaYHz`L+Ezwr)A*pRzr3%^IC<+PO!16dm4Q@ z-1D9m323@;U4EuP*Wyy>(NY+4TS4H7F@ojt0Qouu&j?xX=#46MS48`83c|Cxx;rqS zn@6q!Gz5DNf%?0i6;aq&=)Mt|Bu=oO1<%K-E{$Rx3q;PM4dn zZ3eU3G`94sA>b}wYUvwdOQ(?`1|FS;Ov7(D_t@#8I4pzSr4FAJ5x_{#74?U9$i>g3 zcT|7efZ?_+<8?EW;OSnteE++gXxtvoh2R@NchrDF@20>D?jR$Ca_SJHpd6nhW4K2t z)y?C%L2TNPI!WJ+^SC`AYKSHo>Xt?p;h--6pg(176Fv68E<$o$PEqz9Kj%Oo-6O;R zwhWQ|_$WdzGX3^=S{O?!^f{!8XhmDgM=)(ANdB(qUU|`%|Mw~&L=4#1w)M)(%sEE0ez5$7r{n$jI5>L}?YjVwo?Gw(6 zr{JOdW6*||y?+Gx4(5t^O~d+BW@ghVw|!pbgM;b|{x7Hb8oq(7;4YT{yc1Vz!@j|y z$e`|Z075~e8qi54$B{=L4-^yqNWM5}sS6!4sxkifxle61@Vggr&2k26hZmt9?f@^T z?N3zUV-|cTmD$+!n5;t8)3HlIGT#k!St)2lADvr7O74}cC^eZxr+P$z799YF3RdYThF(68FO z`g8dwZ(sg91_OD_2T(ivqRKteMC0E^F{Cs|Mu7Cu&wgl{Yu#jB>mZ#VM6f;q$W~ z-p*=gkc0B@Ke=`qKijEotoB`_i-&U^+Q-N-C>Wx$#{fHhN;E`#GUu_avMbpEJWZt6yw_#aO;Jm^Zoj`(OEAxItpOe3K8kJ*ps zrH|(luVW2R0osIB<>sY}w!8yLw!k-zbw^Dhw=aKL(%jWMl)!#A1gZbAtAqPryE=dM zY5zmM9p*$)-WQ@O)fJ?XdQMgA8ooh8nj!%W1Q35q7laPWCxNQNA>rIR{MvbO zLhqZ(mTOG$x{&&=klCd}S!X!X!;mW8ZnNpV*=+OBnfUQ`QT_Al>7WhLM)IXEOi(#e zm4tjcSyF6t5! zP6D!#Zo*6;{L^kxfY#a-+mE%_OGJMrg#}!zC3!2Y`cjWs7S^!~N!8Sc(qBp0YUsbT zmuDiib(=E9HbbS9q;ZI zolMNI=w#yO+B&Q)cK3D?OmyUnVd@-zOM(7sDbVD)@DLUK0RGCttww@C>=Gy4W^vf8 zB1WH@pQN$_b7IPRP-32h^J}K)Ez#wnm6}P}fsMSv|1BAqENIDTL(dEzGhPKfX&A#@ z&?pq*v}EIJTBF@vuQhE2O;vr80R^oer+a-4tu0bqEadRMtiay+{9}x)Jscwpn3@)Q z*!cnu^~oU1nfwRo3bCmy`8n;Ks<`xQ!pYS1Dgi@Z?sOwha8+&>Sf)SrSg}-KV&61k z(+&19$lWD$eOM*YPS%!n0JZX|%1Amath!FxTqKm0{6v5FPGzaR*zimRrO4Wu`lG&i z2bEn68Wm`t1Ig^{C&?&b01eyt*4^SQ^Xd4wS6_gF&47oY2@-SQ>3)!cwMZ z0D?@jP?s&fCm*X^4izUyuyE=SRH33}vE|Ss6vZVIV*Pf!V~0Je$F{)TV^Vs;67;}L zMA-U#M}_}OpO`aq2VU>xcu0^+d_C1!*DlyG=HilYDFC%$m#ccRj>3t%6)Z@`iHcCi zh_zU`hYu&;jBhA&pr#Q>&>p9=U<|FU<$=EiXO!?v>L^T zvAZ!s@=>#g&9jf&5C)v8*Kux!Mv2vaV?h5P3TrTgEk6nNph7*$THc6dm6{4?TkHK% zw)X+byE_`kwcYQNp!_v-h>XnFvp3ntKr z^V3!V$57PI&MBuBNvKGGB_wr<%o8Dxi9+!@R@&Yvj@_tMp!dU-kNL*#K0GUx6n?8Q zb+~>90f^%jA$i7#H5R_eb`MxG67$5IMc%G~v-*uCfZrG&GV6e?(UhO*^HN$IqsDDi z5SyDdjvU#+F`y&P0>6~18@*3T@;MWb{S?g1CgVIT6FtL(DMLSV;TvqGBYG{dFIQpM zBFCKPSLyF6@bt@%U1C5h!0j7@wWGojz1RiXY;CyrvOMqoe2byYvz>?IN zSHS*pyQFhhR+1xIyjw_NA?a6ws}Z z-=~A{k4SGPJ89AhCYttuDC3IiJ$^CugYrkz?C*#b(hYFimsH-?oPtT3@A>n2D;HPT z8`tqiVuu;pgF(`4BtXeCVbm&6ab11|_`W1TUZCguB%Kf~hWeVK0Z9sS;X<`$NVUTz zb->%TDY@$7P^9LI>?~;7O2Q2-MP`X)B4(PH+=7OUX;IKJhFL|RbuCvTG=_#;F9_-H z-TPrw?3ifcPq1OJcjFjWfC}_{tAy+xjLe(`i4Gass^DGQ3-xCgKo>nuOGFC;-E*rr z99}21eb~3dysKGn{!UIen)(3RMgo_zS?c|2!XUd0#InQLYdu-l+ECNOEo-l>-4<@A zs$HZF#9gM1 zJNcWGTX}dSAhkq<5-HbK=Vcw0G1#0sQWK{{s*EPt@ zx~z}7>%Q!zAj@!49AqDj@?@2Jf36u2zennY8o31N z*2S~Q#Dh5Xewqn7+U&1de6;N|4Ur$qFI(a4ouEFZaQcDWz!nV=$lr!ia9OEfH_ilb zo4M0x*v}=-pz9po2@O!6X_vi*#MC(YAs-bZq?^YYTKAQQQx`o|;H$`Tw1|~B*ARt1 zvR0|*SH>oV!KyDJ1>UHq^Uy#4(9bUuwGK?5^b_s>@5m7A-%lY@S<41V0pT5vfck`} zp_Eovvv$_rsyXLZFx#U~TFefq;Ip?no9%F&)ctIKfA;Ti2`yy&*RSssyBSuMg?i?7 z?aZe8={6>lsr5g-KOP}-!jd8TYbp8NP@$lx>?z79aw~H3^rU*&{N4L2GY(q1j+g*V zIL+1Yz*>)O*h@`E$Hdm*c6xT-Ihr=$>^xxXeh_r5!3jB2_0I9zFE+f%ZA%(FfV!Q) zO!V?xPRydiV;aB~n=%z>_Ofk)=T{v$yDk!q8H_QuI1TF%E*=b~&?J9_(Nvmcqd9jY z(^xoAPFO7jOCqP_kN%jQ4Btqa3EPesG%^}S*A(*ZcM4zidJ}Rn!+PInC|oiM;ZRu~ zNsx25fSHjEo)^}-wK@ln<5D0%=4Em{`Jn;yUEQ6yBIqTmUG9F)i7WSqxhL%=( zp1#j4esd{k&$ajbC-a-{oKlYEl7LDdt{E1RJS&O9+$GEu>Z!%>7O2c%N2am(ciIGf zbJao^%f)yFjzZ`@~#6?GUpvB!%jq5nGqbSQJ7VIB_{bv6;1 zao}%CYp>2YLVT0qtbq*Em=$0i7zd-vkOuLTJA8{hphkCJP11SIr9;EAZRxQc35vz5 z1fz&|FKl%pnZgp+VPF0~Nc}YuSf=`kQmTKnR{ZaQL(vIfEM#NtsBix1OiS|Torv}4 z4aR>#R#D1k?i<28o88V}i&flp5sTN?P@!55bUR2sxVt>0r5-k)n+4I=SjqNo&mrg5a7 z7~z%Z$hRlWs0Q8B4W4g#2dGlH5W;ewNn%62N2krdi$2h|JS&EW%CBY;?j0I~Cm7nr zh+ma)U_qkZ%xDY%#^(stLTjfoqxhulgI4Hb!wOW3VZ6Uf=DJf(ISAyQ6DY;oY9hqT zqmm^AgURq(%CgSyS2xf1%zjsAboJ249o%ss2#Ph1{AZS;bE z>UJD}Mbv5SURS42nm^Z%G*VuWiLgMp48`XUx{fK?a}BencY@tfDQC;qO*tX|^tm@R z|Ai|NY-1)rVHQ^bF%;G4H@IKDa*OZkesin|1w%O|5@|WBEx7S)vejgJi^wdoGHQBJ zY4#tq>>;|W%zVZ*_&t#`6uN2_>J8Tc|6B8 zTE0H$tk#lUKlLWrtD|KZg7dH|h2>lx+X-yQVX}?aNxNB&joA%p?mX#@*4{ihn#^s~ z(hfNd2%F+rypf~Pn9ga?30g%jf%KQ{#@4m=MH=Q! z_!W|Coe4fSw7OzWM$O75sT!1f-K1WH9>N-{(m=b@nX@3Qi^hH)L>=@l6);PO*6EkM zS1;(aBitGEI|+s)W`V(a3sRh(wz!#713@I=*6L3drml zxm078DFao3{Cv}ro0cKQ;X>HW7YexsMf2<4Q6n=QmZDaM%Vwja>!<)f+veiSq{2;1e^z){IF(QvX(EeN(6yb*AbtzJE$bEc z*XX?c5ucu;0K=Y`3rSMzotNBcFFu$MO6NXAcrb0L^dIq2siPs=IoQXc{NW?Zi?xb^ zV(%gdw<1P)WJ1T2E6gZtTujQ|t9(;x3wu9zRP$xrFRsl=_C}#iDjz&Uv$QB$6 z+Zf|I4U4bc^((}7BsCA#j?GO~GXwomgx)?NF`^bkN+YJj!1>E`X(C(DOt7LbB~RYp zO5yO&C^&@uyu0Z{)7uo1icXUrK?WRmZ&6PJ<&o$zH~cDNtD=xF7i{(2&UluOXUeZ`GsV+zdW1VwiL?3Ey zYZL<(|K6&|sZ**vJ8`Z~Otm+VrUR3ug-bd?02h?UZmXG+H?%46gQP1^*W^pgsl73~ zR2`DqH04|uXlE=R9+oLV$k#Q}Dc;#2iEDH&-%C{5gG-!PQP}BY7f<~#{Qem1&l(zF zbr!(>1l#4{8l3@j2ieyk*~q}t9kN@jF*k~Ev7Nh5NyN!!3--f)2fE*ImKRPmg`|0} zcwBYVxB|nNX{SUe!bN(*|Irw?={BxVlue+CWN^)$a+YH-`4#5MqCVeg;&}~-HO3C) zGtMVB%j&*wy!*H?V_jwtxCTvAYU1i#Lq#p@o9S;tuHf?Sd`N&nStqi0lAFb|kD$iN z5#oH8==Rq(;5w?!wA;SZ2({0V+$Ls4e>^IzK6%zdd1Tl?3Kd9s~D7?$D zBp@XL8G%f~+Fe)K&nhG_LI|vN@4+f4F=}^tDu1^Mx1*tr&U_a4r)g`j>el6pGG(yX zWXNZkvwCxyv!|IKE0cB?g%2~XFFb96U^h3s7OmaB=aDNi-6PuBlXLsJi%dAXrT(lKxCU}S>mgvlnA znR~auq%(${Lfdw1}&$*#D1sDC=T&hlzA#I{N*- z45La4rz~Ctb$CtU2{lo8(n2_kY8Ov}mpDjkL>b#Z#4pn!*Nkx`P4|Hq#uJn^_QMAW zTPSIAKdmHG+J)@BUK$exA`$Y4O=9*K*`k7D;D_kO+IM=jE@EBMXTN@)nk(!YKHvq3 zda_fl^EeWP@Dn0?;0KVO(K1&sq+Hbqg%ZUfOC&O_!l0x-zWzzXZq9Rye;nwS{!#8{ z|F4k!R8;5Q9?4(Hk2m6D0?5|zOeQnD zy6PXA=}o7eUN-sPK^T0pQ5y6ldyJ(=S}JpZlzx#&3_ptZ;F1`R&*vafg4RE(3!v^> zFhmDhNbe$phPRB($+>MJ{a7#W@$ww3o+^iY4U7N>Y)O3%3aA=YKn=I;aR!}3&6Bb> zSkb0k!7!(>Xw#F)^V(Esq$S%qK4*=kJcIbUfX)XPAIW~Y`6jz;cp4vw&pI#UVPlS6 z=E(--S|_*+S&hxXq&e?94Tz3N?}Z3VEBJ=VyDAVMG^>s1S;gfrr{30`)0mBsNgT)D zhndh>MD^TcFl<#~_vdby#y1+IJ%^cc11*jrr2fLOcbm+1O0qWKS{V)N+Jlr18apX?3b^p9;(RjCIX+1OPm@5P9;=D*82`$l6S4~pqG{!T zkr&~y2{GcPb|;L+ta>EOP}?9R(v~{7n}hS z0?bUHNRbt!taG#04z9LzMqcOl0(V})l*|k2T$jZSRLMl%aHf2wKt*oL{O z>aTQ9NIo#5_#kbg8k~yjz+H9a#eE>>$t$0|b(%YoDp^UtbP1&eIk*re8`BI_ui35H zc4a_E^hDWVcRCPkAVD;DRMS|j>P#4$CY90#6JD8mUmJ`++(K%&@BkQ_aSGLJ%~Bn$ zq11qyt#Zqyk&!ZBH#Q6xaLr`p8c}WRnZKjuS-j)%nv(}_2$URsqagyJkI4@OlhOVm zkMGTj>nnTGV!WKSkJ&`ZfI(G;lI=~SO)_&mZKjW|(w(Su zEp)KHpSAS?jHFBgng}^Ix9BQzk8N;F*w`qaw`2wzr0((3-6t2E^LtW67=SYn;2D zsP_8n`E&Rn*~a0nP%&)1ZGZBRI}88&HJRR1{Gfcxyq52MJavey z2(*p@h;DvylpIB5^&F^ZFDl59%up(cngo-i8sf2PN%*NiScmYyEFRDl!s~?CPZ66y zQE`6=arS8m2i9;g^eW#~99&YtnyT!hedL_lpY2~0rwH+L?8&ZCdXh^tk0Ed016MY@ zl%YF(u0g)m27hk&MLMsmgxXvX!6Jms6~z)&4tu&M(#YiWctsUxp324NJtSRaZCD7V zI6IrVF`WOl@oBDi!Ug8Q^LeG&X+z)s=WZ{ww$YGVQ?#ZpZ<~cNc``_m$GrmmCG^Ts zzQEQO73TF$xfxfVBAPeykGHuu%Z)%2D24(Bidl@60xgWV!PuA`H?^SQ5p37wX&n-e z=zS!mk=8}g{@^!9+_t#Hj%*iFRd!iI{E@;7wxiN55|K8KkZA`NbR@MA-<3;I~8DluV#32I9jpubVUB=j>DsO?$!`vdr8^D+lj~VJH)TdYXk1w`SI`HXuD|k(yuP+rfhHa{8AS*)V#*$GL?)eAxp&63Coh1yi zg|{X4th4j1WDc{XmZkOm#UF?3;SyDcGdOaPFiq4Kr$pbdxjdwxdAwb562 zXSp){pr6nU{ce$Z(C^i}9|)cg8JXF=-TQOAc%VP-8276$)CJY3$^cY0>b$&Z`_!o6 zCROfNe9J;lKLVE8C|WSjgl%jyC#=ms84KyRJQU7np>p*3`u|F@{?h|6cQ-a7`S*-W zgaA}81A@qh8LK8}IJ$gKW3DLzD0iXkZYFWj50 z&yRnN3GoE*M{{a7j5-)*6nPJSRIgvQ0SU*09FI}oZdD%bns->lDTst-y?-8yGkp{QdT zT2IO+j#xA!+|Zf;BQ>FF+E30CqYdll@`gwmdR*lNH88|7OJ!#pKDp`+5d8H1@I0$6(BVt`-t_gBKBYB|XbMbnXfhr+6J`x7Wy^oZpwYo=Aa?s${?%Z(bG{N>NwM^fUf+(sB~ z=kk)-4J<@dR+=~(eaM3l<8>ZVbEmAIa=P~lk72V1hJ0DC+kM=(c6>K|w!U#KDOdE{ z_CdewK%pBPB)Ef5{~q=Ta#4A_{X9qie1(_szeX`D8zW-?y^OJyfw8@+KEUaZV*V+8 z8vl#;MMh0n^vWXyPL;^aJ8#d5U4nNU*2My#s!-6u9u_$Q`hV+UI^3;m-+q0TA1L9r z>haazhsn4YpSpO$?#969TjAS)%!)GHNDioBZ;mSΠ`%iO}aP>9yumW1a!ct~f>0 z2B`bF6-8_At&mu>j&RWVzo$)W7KY9gB9lq$2FZZyBZ#nnv%&J>%kMit&h4y|HOMM- zLtW^MU*K|iHJ<1}rO7W3Wq7>^=cfsnG>j7?nEzZThHz}A{9LKX>Z-EIjz6&q(D^)S z;1Kcf7@MP?KZDLc1|5!n_@VwtR)2A1lE2jwAsZ`OeS7`?#gmo(n$+X1e^YM*qMSp(5ctay>N+c<*19o)f`#qNZMne)tBPej(9P3jwW>Ieq+8UICk39>b>Ue z$+>8!LsuC(zT+Tdv&=F}&*Up)68RqFZj#KzxsQ19^cCQiP%xg;1QgBG$u`Jz3ROPF z@4mC{lY@h)zEli_<9*DGLS$ng4$Y*B;IrOZT8nxFn(YFg{^R!n^%@lflREu&c zDJ?2VMF9b3=oyHPl{k7GcO4r8QyR4HO#71_6cE7;*lfnV5xlD$mINq)Rc_yjgkP-G zVHX80FOt?rNgPs1a>K=Wfp^+7>w&3XR2i>95s(cE zgibhn)^15MgsWrY5#WQAuf{p;sAMVQzDs_&;|*%HBu!$U+ke-i@9R)#ab)+)=5kbZ zSh5cP;K8xKEmE2n#Nz$nKq=*RGryKjD{YTae*@ct#)K+5%uWce2sN*}`}zfQ&F55T zmS@c~1jDO)T&+LsAf$TcF-jOFTP$P)M7Z@U2T9xljy33S<`$WxxYFSl_>u2{zmI4x0XWd%r^NM$0m;@d>VTOAx{Og=4|MGSAAw!ecUwYhSz*PGH~ble>g_ zBV;X_dzPZZ4e_0+pwElD1i)0`#QVc2ig*!DhTmRi8+^YX8*1n@lmNiZ15xnOBm+kq zbnuJwbtlGiE$TdUe{1C#j!9n+HP;euJ6_mmk6i(+;+}=sZem+3P3s#diJ@`gh7TW3*B0iM z?m}kGwrA{S9T{gQPRf74pTU$T2kEP0hF&I~xD&z;*iwC0Fe_E!J-B?=kWK1?6tZf< zi&t?&eVTJ-y4jA=KgytJgr`t?DlQdfAqf+43;1&9m~|U2K3%*ERk?3|>$fj)3@!Qw zF>Z!KsP8Fs9d1t}PtB+oPElqh-L^wz{Y`ZXN%A)-<1R)%_V-vbPcbUrhqzefhzgid z3Z5}aS+`?shu>zH)M1soWY)0{R#?dfrDH7)9nQ&shlHjV@qv6MRh5q~mh;8(JzQ1g zCCXM|(SLPqx4RN(r9UBx_m2>z{RfDO+JACu7aM!azcbUn_>J<2;-?zTOCL%?QaN4{ z0!$PWkgJk`tN^A^2BZmEsB)V2JC*0uQEMHSR_6sd34d!4)H(w#No)O?|JW_lha^G8 zF4p6s{i4fZ3U`*@+v^Ex3+S9Qwif`EY^xDYZKo|m6lAHXy_ki)GD;ddBtyHu(}Fr= zAY+OL)sMCwhjK_oAy)OPOlx+vrU~Hg4sAVChp(A6dl1Borkp?@L)oIc zyjF!cd}|R!%)42EG{lz7TTYeTz7eldT5AvFKfyu&xv^uS0XlATX?Oy0ME>1gz6x;`Ptw4me3UAGZBX@X8>{o%d3k87y8Bnk73ikoW`2+|WF0>7DSPWq5dj7D(jE~k zL72#U?Kd6#B1{Fd6vA4?p@hA0lMH2e{DS3ptTpIP&ea_Xs(qw-%Z!~sw~$rc>iOwX zk=A5&_%V+GlP%K|A`AqSerYV>jTW)|LS^+*xUa{1NOx2%E})+Lqi`25b&}cDu7ErfavA z-UXkz#uO&C?BMyUi$B>#AW>9LG$YBzLcwi;T;d(wkD)ULTX*O6D}UM`qrRgS9*9Cc z3h97m@Rp(Vs7@Dx&WDa)$XeXW#sKarRm=e{4%3?2zoRl(ga{s_GIJn z@ZU(A$bz}0V$$4dIhacX?bbu}^-aK@hNQ549oM3&m_4nimzC^RJjFm%@SgohjKT|# ziYa9{r@Uj7`=E`rc@Q#Zxk!);rAq~~2dh(K8ij-5Y-15rMfh`_o>rEHq3gA&@ zfD38n!_-M)ZHHr-A`+)3#B_ekQgakLl-2NJD1QZwMHW=}ec17h-8PWwy1y%6EWq9$ zK=bL4K5BVSS~uqi&$F!Xv%GDUDYI-eyZBw zb85af*??+@Ol=jMh9yhx=&1Up{%P`y_^q9FoPFjXb%cV6Xac6@2CR%5a`UH!94pw) zf~hcHWC_e!g;|>K%43X4j&r2v$qLo$U+^{eQ~>CGW{&=U6te&OywBgI%>R}-{HGmNFYs{F2~`6^W%jQyw4_nB)E1X=iIyR=0p~r zm*9{25iV#3)CP6!%dN~Vle`X7Ywi3$ygt6>_+}bov{VPf^71g$2E>Ir@vvM$q=Suu zQNiq|j4Af3s0qWApYh>4h}r)<={Ua)#6Fuu5UP44Frw(t0vQqIrKkCYNp<$kDf4~! zfk-UU^LxoOCdiN=*9!{0EU?qHqHyPyIymnQMAOh^Q3l8HW*_=*B~F837RoHb#e2^y z3zkC&r(teIvryl$Jw&xE-Kh-BOnzXD3iRj4qTl)y!%8w~8l?hZsu0(z9mxp+>}WD{ zq(ZgY8m^_4%xW;i^p`@wY?bhSvGxO9#dDOCu{M@KNORXZm;qUvnQgGjw18ZbZKx_u z^&sJdE7f$Z_YP-<qN3$ODW}%R%PCP|TA9;3%lXR3eMxaK zm2LEU3+XfL+*3yt^tU8QWGyPRvx37hLOI%I!Fpaz(_yr;W03KUGy>yAGV#^#u}>)Z zYUbleSgw9Rk*YlOvmTiMFv}JJiita%BiCxZA7C#*Qbvlmg+B>i?4m3lfK>EFotTRZ zTSQoz4PSBBBE3#XPOv7VS6C7FfBpr$;qv`loKN6c|0D3I|ADgnJs%|KWDYR;clDg8 zYUzPIgzCMhYaZF*V?`L!LnaBNp0pS8g)p*`Ya2@(AO!F`nQqXLvh)~nSJS8@kR2{G zKYu7lE;!r ztsD5!5=er^jDl?1H&CVpz7Paki6tB>iG_xI8kE{M+*Q>-vl}YI+Obb2C*nYO*C`+9 zyHF!^GK;pX9(tCX9x6S_bU6CJ4r=Zh=}ZSrni@;TN@VsfE3E0j13gPeR`1W;DR$txTIpAD@kD(jy$X%8zBD zxLk=vt*TFu!9`?2Fp^Dj7Hq;o=CQ_Xcc6<~3-eH}oZrK_K*IJFII%&ia8eU8W;1lnf zTJrTgn4QY^u)J3DF5_SGPG~fxV0NWL00eVrX%L#pC$YL?8j1*nEe0H_trpmRjjyTl zaVBk2CWVI@aWbIoTI_f>70~pBv7BSy$+A?pnHzuE!P+RSP$W){N~_e6Qw>(K42;%e4551CI*DP%ToR;)!k{nfCSKr==2e1B}4W<0}AQykay8fH#k6e+nP zKBY>#2@`H3=O4ffjovCXWOEf5w0ZQ%`%aMYk{`vt?SM;et2l(R@0c;TThVetRJuLI z_3lR>IPMs!J@SD##jGl#=Y-Smty|yhbVJF}VW+`&rcH!jsOeO-X%7{&io4^q7*nFp zz49CDVx4r3zpH7jm>KcG$e{;+$xNrmq>N{Wqsj(kGnlRFT2zN+b4i4nkxSROjEq^q zrMGLfeZqA!VO)eyLsCo)bzDroDYqI?xuKoKP_z{=Cu&g{Nw91L!N%U%_E@&VyCnK- zuFLh?c10ag&X?@L^le&u>X}nFKD?^gK>n^TbpaQh=v4SfM&uOra<|87G{BtBFyOeI z>EWq=SXd)8Md@ODj&%f`)iS+A?eyWQ99U?npHT{XmBy58dIWJgz7vkr0~z;bapnoW zp@xH>K;1i9tbJm{AA{Q(w4{8F z)>hYg5^_)_yDq%=`niXzD#+cVG+#+`Ba~w`9)M3ArnD3`)cOpSL_Hm8K@REgX7l?TW6B6K&V*#5nv~Wdg0`i@ep_9;|c&W}Wcs$dT zp5=(J1Q$2urOcXhn9#pgQ{7OdTI(0)FOzbEBE&+8C|BG75Kk0Qb9tjV9S5TL=i&j^ zgTr5pDsa!gE+|lnvFsq?#7DeoVcFDls`$yj7VfR-;ZEy``(tGGXdGje_u;`nLM~)! z$G@^#n2@Y3Bb-a7ypEiTSKVNrw)y(Pc}DkGF0F^iXj64`fJxWEk!gU(7xXyD=*2lP z7@UL++c7%5NfMx>m+*7(xUjNafm}8}l*X0Kz zDsxPp0QLGodEiYZHu&*ZLZv>!XL$T9LQMYg4)p&_ruz3B&Hs|AP}QMzp*9MWsA!gG z@8U49k^JLQ|*`kQ?-rw5Fim!%@{$NC2k{qeiu4Q{0{reG|sq|$f#CNp>)_Fdjh zO{Tm*-tMq{$nZXus&@O9x#ttUV3B(VRe*&MNxvZNvA11#RM9LE4ToHCLb0 zH_-VbRUO34_`mfr<$!g3$Wz(@R~0J?9UfUyM*Ws%-+)8aE01KScnBqA}+F?cvg zPJ;D4Msh3Chy<0mJnp^V_9sd)Ph=$LEY)_H!AVP)C5l((sR2!605-TK`-qE_d3&TX zAhEV$pIbebGZ0GEPQtj*M9K5-l0&`j*+lV^`QiSfSjG1b%um)v%=q()fw`fwo2~JG znv)Kn?G$sP|J54E{2ZY{RzrE06=y;am1m%!z@itY`$pANqG#^wp8vIgK=o{Z5VLQZ z0XYb=Wa$z76zfBc7ye+BR>+I*D;S2GKw<*)UXvkq2tLgo6{loix^8=^p{?v~j z9>7~nx&gLWbF>_R^g`nLWWwgEJsEx;Q2Rpi4C@iqza0d+z5@omI5pz>{1N;f+fd=J z#Cr^no+ABh_HMXAH^&a*RdM_m!enX22-m|Hn4la%vPr#miXrd#j|hrbaA7 zb*Y*+CU(8HnY#9%L&Ky>gF#Bt5{t9cK;$PRboopH-|Nr9f!3q3Q+_+flkHUbOywZV zTdmfmzRYNf`&L<$!EDW(Ro8YZ_%^o~XnlSLKn9^ET0})DCuq9vm8L3qt3+L=O*x~b zN+Mt~LCIIR^Su@4Bx%wvlMMc3o}&KIH#JdsH!zE`$gL7@hp)Yj6MsZ44e}P3Dz1AC zT!77B&4pkWT$C<;0T6wf=suiC=H$j^+%FG{gYml*q|0T)UDmAx2f?!#9r;> z>tfn3(xtq_d~F_S1m9)&ua^BR4SJ}j4c=ohtIZcW#WOK51v@l#RPJ`7Jz-G5FCZ9*1X~J-g)q+ZzqdDefR+s0<2AbevU&`gqVLnP$+0omOmxD`0!7g>NTaW;Ah4 z(qhcxk>WiKcd*pWrKUvo#A0l`6pMX|vB8w}I%<05OFqQ5+X!0U2Am=m3YA*3lp}VT zeQz$#gTp92bMpwG8!3ZnIeGdV@PP53b+%fy!Lqb;NvpmoDL38z7TBck8a<9eZ$?o9 zx=i6mjbwe~0%`QrWyjR7vHyy;%8IsHe9a{;78p_wg2aKhZo{sMTq9nf-)%BPRR+>~ zXRrTtr7W5MB+rAiy@5*uzg?v_4U`(P8*}18vC>>5}b`bqxWHqY+%4ATfprW#F zN4PahX+d*?=}3KTidO|tUKR@RLqqnaIphPz9$f+Inop{;n^UrRYZo+Nryb#{zxwiR zJQe8TNr;!lBw)-#mSo|xnWmcws7d+4__N_8D?Oc!F17A z{T~<9j<5l5efhqwE(EpEAJN{mZ=3u`wId>@1ZajDJliS_Dz**lBo=rw@fDKYWj%#jGZO z|F%Uc3Z_f^c*&X&+-`PmwyYni%~zSgr3@E}i?b}SO1JW0J0A?b+9Eq;NI!!o_Znpg z+c;_&HO|nt}9$wDZ<(qs}Rh1>V<1Qs&ft352m(%zOO>6JObSr8 z(CkUNuSvIsK)iAfDi$ShDNHlR$%3QvGS#HL?nFCUt+NGHVJ>82n&&Qgot zV*e71P00ICGFITX;Z&r$~IR0V(UlO1gRTS zD-;n8N+_&B#sp~)S#FSD&^N+V(4bUIq&uco32<^|gJeSeQ}F7=H5n~5%`z3zCJ~ca z+QYClk(Y+^b<0z)YM&;{pK}LKu6pN6&pt<6txjf8qz$T12iZ>3Pg8DPci+35j*e&u zc)Z|&lDB{0amkN?quIEoMz12T*`yjl(2OA;t_cW^$|iWcrVbf@GS#Q5!8o`3`fzFc z41E*D)h&mye#O>Izz1_ASeRZHX!nhu$%of_Ku}cY7!5;bS`B+}@IzqRe^H{_SHpOq zPOHrz$o_OwJ;Buu3PxVq@A1~2h#^C!Z`@<5I5I+cVvJ(CYaMyVL< z^-Oj`s!Fzx!n3@V#Md3?6`1LfT9Q~9X;S~X1D;$z3czXIG#)nHpxseYP8yJEq|(|gZKm~1ucuO81&0>H|#bdyRI6PHyv{Mp6TQ0Oz(4(b@=Pq zCSH+4p>aFs8TadKczSrFJhY`LHX0$^J66Yd@g=*l^WTgAN~!q?&x;w$U>?F}3xW7s zYf)UUrk!zxKQ|~*oFRyi9Rk?$$82>_za^%DW`f)u1tJigmF3n<-TTlGU8NRI6KjLr zDxju<>@QPPyuVCQ^C;^fKRry_Rldu#V@E@}-o&d{3Z`zg<@wAf3(GbpY$wvP)sEdh zCxJkOnvC0;$d_Kz_-|ngNIR{r`*eOGLxwAb6dAp_yI>Vo-1w>v$Oy;-d?UnE_{&Z8 zZh55O(@3>g1&eTO9YyR!>Cr=)B)qqre9Cn(U4C>P_n!2E9P23skD^F zT0TCTNq9}O4@*8&_es3T;!CI_T-Yy32VoP4fX29`It`yMjdW|>QgDql?XN6~Gsw5p z@9<>%du~26*vB_8&_5O-q}5(A@*Tq#gwtitpYbIj2$ zSj%#k>|tl1N(c3cZouHBlayt%nY;Mx@4PBN(0SE2s6|;L04O1s%0Wqq8nq@8Hp}J* zu7jP}h%a&^=)Y1&wxA`@SQj#DRzN;r5QD8-}P0f;9 zT-t`Wk4e~xZE!A7I?LGm<& z*xKuQzHb}Vl-E1RKNLy~>(4FJTP>(kTm16zZUsylQ9WFeSICTYORW^$CC!Qmbzb5c zxP(rjIDyiC)?xqYc**4J6$m0+QdsapqVXAAwd0bSI0WGwv<0;Q$jWCuHP`gIVv)(G6-GDkY@ zF}rmuH%BfRevm+A9*xaHJ60|vCWie;Q&KCL^@rQU;+1PsDF*pqt5;ttouoF7(5~y$ zdJ9(5*_zDn;=@B>Z>5QXdYLiAw-K%6t{UdSeHRN_;)loA_rq*}jzJ%P*=&^%2Rmo! zN2mXbKD0Vg7v**i$%eKl zP7OC*V%5WNWC$2lhEMI^=NAK{8KSV4I3}=C~T|tw);C18Frz@~TuXBxIRg z4~%s!lJN+$Sl9MKmxR&{N0g;mA78=kZo-U?N8xr#Z7IdG>JUee1H;YFFo#uq;8{`u zD!CZQ7xpDxyVN5)f5tePjV6Tqr<2l(LOhh;BFNPRjNy3jekyQBVX6UeW@>2;;c?Ww zkHk>g73jRZCc}|^<*b;Ywj zlbe(ESsy#eIU?FDXrGC)nD40c*=@PjFIc5TrV}sq@kS&LZ7ZarUV4-*F_&270Xl-p$H;Akg@jolHW zN3OsgiSZ^9#%XjP9I)&L*Nq|j{DPXt-66K9c8IXH{62pcy@R~pZJP`Wf#moGx`TR; z;xn@k1a%pIZTp%S^e_{bdH77ZR}zPxV}N&qTr1T#J@&}?+7bD%>%-*he}0RbnPBiW zNImGQtiEQMq>1~N%ap4xSOfzkP&n*Cf@8jiQBlZ02m05>v>SWKM z0LoZXD5?Suzgz4BA`yD6oJm3ReY;`#0n23G>OOkXvtu0~2}<&J4LQQ)(J4Otu!X(f$}H+!mF#os*O$*p6K5_Y zred+f0Kl??RNvNHV@SA9F5zh-Ina7vB$H3A5)=(eVK3`$&D5W(m0Gshw^f*29r?!H zk#j;@X^`v z(j#h}dAy!YRABK2>oqM67tLmeD&N(ua{^-RC0P9&4x6geK7|2HWeq@rB8Yl9It==Y zfbH{U#6vc0wAD|=@#n;E&3c>IupvIEaHoP@7rLzXI78HGj+0zTm)+>W-IysY>GPS5 zq(-*D0ZJR8U&BCKjN5`*X1l2O$zf+SWDh$f$qJhl#Hv9eCc%+hLS1D>W77Go>hN)Q zOUsHDj*|V&Nsj>;nt_ALk{D(46F4w*G)nhbD-q1d6Nt;9-I;|r6a}%PIIq5-AvAB2 z^p5121$0h7_CqPVzNx1!&5bAP##X6Ime@3{@guHQud=hdHa}SQm~p}4{HmH}q@&5N z7pNX((#c@2hom8=pWEXxM%`I|1gJTd;3hf?{egFdx^!k;GcLmYkC9o*BrI9xD)k2=Q zvb-+Q{|oC()~%K*?4T0hb}z2{Qd}NqjxW9;{1-aG+?b*R9h=mJJ*(5NgXg~VRYSB- z?SIm_CiZDTm4t9l2bnQfDhRc+yhI}oQ(h#EB#>okaY2)xl|y=jB0mWDaTc`Eo)3Q~ z0mG;3TG~Q47@z1{%>2ktf3S)LTjv;{Hs0xPfFm->YuLbp%sY<#=)lHuLMPC=YyyZ< zD^Tl7&-KHqGJJrw+al#kjz6wjqxr%7z!-4z+&rACrz*;gNGu$yJa9xWRO5C4+JctE zv`P*5NgkAHzfFK{=$GbxJfQrN(u06i!8}+^6>8<$o-CHjLe&CnmLK;x$`BT+N{@)w zMcjeK^v~uz8AAx|IeC9Je)yJn=^0@58ENi2)V%FG(%b~f==cDQ zhJ5TUP(AWtTK^||*e*eM+9DT(-u^;w^6Pe(F3GXgP>}XHL2g|-?NQqmsF)dO#7?o` z^%d#H2HEwUmfC)!A$SVmtLrh8p)n#527C&LITtOrZFZNImBRjS5nix|#F4!`K}*!U zQPkzPEa^WF^CC6B{o)3P@#tZNPBPkV`gSz)M4)()s;x)(SVk)Kl{SdaoU|rlr%%wn zhkY9=pO%<(2k{_w1yn8`@Qr4vPYFe%C1ozLzF0m zrmnCdb8QAmidvqyfd}3Af$y^4r+Ybl3NXTq$Q#Rt^8mm$4&`RB$@aN)tUE3mu+=&) zOVlAQu*oL&#+|J2jXEkMH9VYc7bTSLe)=~8W1$6t;#05~7Oy-WlajLsEI{^+M30ud z3Dz~4O`fx$ersdHpmxR2#R7M9AVA$RPd9k(5^ARs`6eft?8o)QfrXT5EahHtGd%G-2V1ffvGu?!Ey8db|cwD#o)J_&61HudZg`75}j^@ z`4Nz7RQd&twO@r#5RYM0!Hsb_*i^4E3tk|;#WZt;G>V`0GW9$VaBbA5@hTos%1=nw z-kXJzad~6YJCmrH=se5{426!3j2a0!h{HLGzFl$PC9Myt$e7n7>j3iZ+%5iI{?m4R z5*e(>u{+9&`Zivb#l4!Y@$;JK9)N+lqk00Edj{JFSnysoY%UQv0O;WA>s7Vko}+&R zR{n$3i9@F%X~3K%^q`}E5@T6295pzB%)TB{vtGa5ZJQZ+)J^TyaWU>H<(PKm3R2JN z2(N!;k&8w&ZQajeuF`av98*D(ki|GMUACBB740kO>V%=Z#`4@rqNtwaqmZeX>thXp z;1lpN(+RS(_J#Lk`(M0Vj=m75;@xmpd}y@<-ylpe7o#(NAO95h2{Sczhj`EOL{pC9 z0sRmdC042(rMK!nx!=e@=gn_lw7J%1p&--;#;+$av@#ceZ@?&!U&1&*p3_^KZTRrF|7yd|y zxh)S2LlCEpFh~XPaSCa`^3gm?wu|@3zw@E``@=5y!wdE!1kMQXuj3;`>6L&j5{vDX zK=%5>3$+MalWuvIT1@B%j@V9dLKtK;E4lp`W6$vi96F8*`T;h~K`6LSz-m=NasBLs zb6S{1YwxYk7-rP5WtK%Az&+js@(f$ej;WkRx&j(b^=R;;xu@b{31NQFb{cA=A^R55 zPg8bo@J9wv?hB*ubqA0oO?`EKHwqQ8m8`BzOFKBJHVo$qQ0odLvpuhg0LsKrRI!k2 z?M}yFXrD*D0atGip0|$xjHXL#GL)s?oWrlyGN8kXnb3+1v~G)^W@GUzA{$5zx=zBtlPH^*hdBe+F|}v zO7~A{@_(dRN=D`;#-9|hf3-I0MAd#a45Er$mN%BVI3T`)o*LS_x*>zYTv*^(NlFvh zymCPB_rxHhAc6k;y={NdIlcV$ML&?LKZuB=(8ndNaX!9g?KyLpAH(82L^VBeU4Q>^f$65g?9aO6K2BCR+VZv*5bzCjgCiT|2Xd9!_GssAzx)H1nG! z^nqfyyM+L8-H6bfIGUVn9aI%E&Y8;&|06tUta)kdHi-M7T`Xd~t#~=I!gbx&Cs%wT z4eRNp`s+`iifsdEFfcV`R0kCisAXmC5Tcm6a0}%*c?ThUb~I%)T4rP53-0^e!MC34 zjY_i&*oq;B`khw)8lBlX2qL*{mlf*Q$ERo_vP`0%j(EMX&8}xU>#)Jh4BXl4P(C^B zeb&*)OIWCBXTgaX-usBRsd?rLvUy2bu-9iF#irrcWg9*)e?~?M>`*P`#56Th)h#tFy=4LI*cS#W{XRT13?ZK)y12#IepsQqZcnojyd-WwHm4xn^!Yrv z+rj%OFEM80EbWkirSEaheBNTqee4jE8SW8V(53L%ghb@wSDtaN{x>*oLfyKfC!q}I zrO585uN4U-S~=<_zQD~Jf?s~802!+L0kGsBZqxf{>l~|`r^jYRu@=4@9`lG*13~ZJ z(y7(aUZW|k*S>x+j5>FBA-MJ-#fA{~1t^wY5UGq_61_3T^AbQw?uuv+5x?c7=<7pa zK+cweLMa-53AJq1wAi!pjkQqWwk`wPHh@c>;!Gy95fg%ymL`ixTGBt8M z>4$POA?747IsQjfnG{PG{g;~8BRz3c! zr2YX1C5;I_19@RCoE!_8w2# z=C!`EAB+*t6qADVH?%300=T2q--2-XgGSaZBYO_`$T1FI?n3J)gMhG5c;fM;f6IeP z8)*84C;G`?-h4xufzXqLW~v!)8YO8dS|xlRv)3aBiw&?%EyPNTH!YMOQe2Q1 zW~{`AUsNC~l~5gu_*^X%oLTxCSLL1rF!cR&c_jNy3BxfVmB%*hr<=&8KZ- zsP|XiLv2Zgl(2x&n3&Q<7lIUn1mNIIl%*ufe=|xbjE?q7IdIw_*-SvK=9%1BDmxvc z8T%VvE*S=K>Leu64MT4p-?=5mZz>RLXBhfd;l#@*Ng~Gr&_rJJ7no4#B+NM0${2<@ zt!Y-zrztH3O;Hz1ENn#RsQ7pd#T^tKp0Ktmqn*O>xw3c-Mij?fgPEOXDc9&!aF|h{ zLn@clT|-jiq$M@17?PR}M{2cUN5T;onMJkviW%B6mY`KJ+SLk~c&GNlL(LqLgq*ha zYRbkWB+i-R(-JE*I9cK8B8_tRcE%m7o)xi>E!qxnQhb;K2k3pH6*&pg`1)fHQ_Dh+ zJ{!8eD+p7x33mjgcyti&s%WEKW9#ZEXyJW>*Lm_LB{vpI_@z2OQE07S5MCvjoJslZ zWSS;&q?q;7Lwumd_HKOEf-H-kV{E2_Xji6VaL)$ma$5&pYblg3*w|Q+&G;uRjwVW% z^$HM__-a1Xd>?@+Q_OAyp(TE~Wm_(*hg6ZVvm#T-ft2x9`qORm!BKo<5!>};r_N#BL9hHbjkx~B#Ey1Kl75X0#2 zs2qc@IZ7yzDfW40Q(@UEr70$9T-QOig`{;}HPWzCDc8l?%(wFWxbmfoHO-1UP+B9{ z263S?@nbcK`M*>pl#Q|DmNn>M-V542X?+G^FgAx)5K^MdmD1b~+mVYRh_tL%&Cp5D z`M$3{2<7>EgX<&svH3G<@^OsdpXke%h(K-J{o&ohQE zaNORz(gGFCXz=oZ?&W(tZ6NvLS&?AD^MHYihy#1ftzAAT^ajIMuvrFG4M}tJ>ai;O zotHyd-Qjc(aYepu59U)(qH1Twz=Uc;JDZbdkqZQEI+Mx^C-wDS%#Qmo z(|BUQwR5@BDMz+*b#N$uM16kuP(8aP6n5rH%C6UkeEjWdUmy*J!uP3C#QURM;7=iz zgstmeI>U2-;bRwP05CACGtfF_G_WkKb+g@!Kh zm7c>M+YP*vz50)!@BVtHoRV6+=hu(3A0uB+vi!e+EGWh77#~h$y}WLEnSDH-ricOQ z`d1pkYjK&z;glTMz+L`=+*JfkVYNPO2`J~P+R+*HwgdLi&gprf95E3SVhqsnxDfy) zI7#gj`z=3LQ?diz-(dGCSUS_nwrih?dPM_w%`$LYck%AUtDVN7qY0WKFq%nEzPw2v zM5MvJ=H9TjFELFHGUINf6>^_r<|qti;R;PwGu%VHI}fD^=4JLqhl;hNet5Sh{%Ydu^LNI zVjhY<8)DY0d6v>Bc^G(TQ|=Fo5L_{KPtLUzD$`L-SU*oQpS@R4h!y27)6VuA zuyJY@?>BXXyG$l0X=ATS;wVn`Fx`I&hyng)4#@^PP*@VZ8L-pNGJ|BO*WoPaauv^E zuy?9@PR-uNDnWo4I9$Slon-&@$AO-bN9q0}@GhWbc{X{0MZaJnKpN5$Gw7|mw&fz^ zC0sr|u~9C(qw71r3|#+n!hwx|9pKI8yDVk;{*Nbk(gSR~ALP6w$}z-tnk9t4{KR5L z@=jS>^;=SLtLD%92u%xU@DGQFS$-$pLz47kzmaEhWq}I~Tr@Gmy-OUyHhEa5N=YoW z>NRwsLj$yKy*W?p+uqtZ88T%KKj2N`l$8O7vx5fNUEX|+$L=ynE$ZtH3E#mOvI8~?+iS{7c#Ahh~cFAvmPrZo> zTKp{1uO&aC@vM@~J z=E~4qM9P|HWy*8;_ps+|=7iPFRT=FYo;1#ztjpiN%TD)i-@AQLdR$-Y!EM}m0^05A zgY1H@&mn=`i3N8PkTQqe1l_5FN=Sb6x()6H5XwfJM;P8oL;i%>S%o~Yr;oghcqn4< zCES`8sezT-2pE6N)BnyFiP86{fT`B!&KOAmb6n2)__;N{R=U1s?sZYA(1qUd1l6(^ z;?YGn#(w40MJhKVc2}vGA|dFL20ecaevD0A7IfE0Y$XO<$5S3#<@PBXO79j#jp zWHl-7vgeNzlNibG77_!z!hLu+JOhC7hQ}o64oJbN%U2udUn?6jmLQY2Dt_e32@KqD zqx5M(3ob0k9idjxUKU798P&7dEd zrpaceRMrSP z-V6~`a^y8n6ucrcYsL$X@lExa zqX~+K`l44B4bMHpC=aNHKSdI+4mN@HSjH8s-3L1#Dy{t24}6)GHp*YWr~+|Xt(BPz zIGIb=OR31oPBItPgR@>6URs1QNXp z3=$ZWas)t4k%nY^KF*j#oYncHVe%#$Mv){wvdpCx9@;-1Y4IU<K4lO?jr zsQCh4R7kJ<0@{yYWV>Cer{qW!xHdyFOYK&PRkNtyfq8=vlv8U2AIrnLp&n&aM70Nh z29G1``fjCpXO->}JjxGZT#2CR^`)pp8gy_?<_}0AAL7w41qpRYTxRE8y!CK8a8@5# zvA}o0LS&!m37x=x`d#ewFE%^4V9=hx%Ex>>E49d495>h^&n!t^B*tybp%ygVf)-=Z zrDc*xK^+Yt;o|Zw^#7R1y6vERN=lB!HD-#O5*RmRWqOf8FHb|96D`gV*eeQkIBl$i z+R=P6U<{b-sQn4%pi_-e5L~XJUBC6XF<9;N8il;w&dC-iQh8~3Obgt9*Na8#%e?eV zov4m4Rh-uZegml^aMc4hW;1YZE{qv5BQh%bp8RErIMk((b!JMmLR8ris!R0!)?AR; zHGxJ_OUC;<>}F!I%2}`Ow<=JsEv6a-Uizk%S8()UU4O?cnBi??i4AhJGWfvqLs(CT zudPA$1f_koYZ?o%!if4n3%h$OUmvWlp}gafeY7DYID5WFGh=MS zW9X{`O$gmtT~hU7x;D~q1ohJoMbHmbsS9c0OX1NqvX>y)WenEsXe_`@>wEb4;L~Ws zXfIaplf-D5)tsFT|LrX2J*ItKO4pjIM*>J4E}ck#fET-MhuqBitM;bE3Sw!AoO%Y~At9 zXqx4@q>Kcq>u;X;8i>;Iy>7RW3@wJ(3zLT9H8ZS%F%5^a>f-9c7yQvCnQH9^unDia zTqnlJiTR`pqokBooqO8XGfoLt9o#pb7KLG7%*P7SDluD@xB`42DU`DTStX$-a-PJcXLOO~|U-#V(>Gr1+hP+e` zAkVH&J*o{s-N~FBaGgoyN%qr9Vssz4dK9h%;u2W16Ik6td9}3P0)C@aKX;EUu0Fw6 za=#Mt(?V!0PhLEw({zu6P(f41aKELgGa`(HT)R-Z7nG5x70Ou*LG&e_1(z|h*{pJ*v6svWxT z8=}bNW4PTb1fBFAu{su#a1Ao8v{~Z`GV+qniuEf8%wKtYc4EGo87fnHx5JHAZ84Q^el%gvLSJ65g|g4l*<#NDgbZL7VLU|byILDb5*;;`D=g-9%=Uf0+gBl z5m5Fgjj+<^=$P1i0)77(o?!V}*?tDpj^$zrs60A4e)(D;5On4rFacBnc`9xc>Xiip zqR~o^bmXMqe10*49iX5j$=)U0-Q1u4W!_#8_WlK!KGMXvqN&JzG&T)-h|d%yR)Y36 z1gHj?mIW8-)JnFWaekJCrISWg(R}4QBS`lm>kdbwElD^aBFHLIp$Z-CA<^)-NG*x! zJ)S|CTZ63v>Uz?t3cs_dIGVwzvmt7HvC>vbtUK>U@WMC%2&C`zCktFfhKMpAEA2QJYj?w?_2Kd*cQ?XIQ5kd7OhuSX!Ij9sU ztjueOMU>R6P$AOyM;Gy!FEqFCS+xe#IImW|5Ff}rZt87g@UJ+r@QO_*Q5&`~xe!e!`y9t4hr_hN#I@6|8?yW)gAz!a+pcq$oeZ#C%>E%}1`5 zrS(24^1T}LQBu%_Zj#gIN*iq&canwDae{Nf8gN!@u?Q_jZ=z76-H({8!%?j6X|Smn zJs4)e>yu>FL7r(-#~5ksB;Yzc7m3Yzo6sL_kQz;Fg!pu4ec{lfU~z#qwE?-j9AMOf zvS28qX6?ne3?}ojD*AMbRm8a>+8=AX81d>oIcIQW*xJz|9cnC5jupPAx!<5j`<3?9 zj(XhYiM_>Ob9Dgxe|GD zXIf0mR7Qp>rp!!%%^G%Uj7qYHtzvLoQm8fLRd{HOnq-7(XRpji5VFkyRLb?9t`@Y1 zw{g${kcd8Q>17N)=78s9!FA|};>Nw)<&6g8q|G0x4VO3pT8thE8`Mz7oo449-E3&No352Fc>NIETG{>(# zl%O$!YbVxY_7^*&vUQ77~v!TkVr+$01vOiB!Z-$bExM z`@#O~E4Du?e@kZgeQN)^$h0(GC%@m3k;}k$Ve3<{ohrk4$)wz`KTQN>i6o4$8F%I6 zgNSzV2GM?x#pc{*_ORH-f*FU+;Wj*==TT8>kOSF~*B8jeSVQnPd)UE|Qid7heZtwd zLRU{$o$x8_9v`KpT@`WOF>V(eh$6dJk>q5=b&ve(IEkrAiDLkeT`l!AU;5Af7zvl>sZNUxfOmg5hn=g$RZJA}< z#o&ekIMG2uq8@P>Jw~}ATo@y}mlbH+dic7w_q<5?DsDr**vp|vr>v`Sp?VWW{yazQ zAUb?8*5ZgUW0<(PYUH^cc;u1F8$$>V%g$7KBBPyuhsf3oV$sT5v@{^Xu;Dwwc&tkVLu4evZgf}AqQLHS>&-BVKAPqJF;sJO7b)A(yD~A2CbENP4 zfWh;;*@l+WrD#vsxMC+ero#0;w(3`jpNTqK&7q@BSznqxwV@9_Vv#NdT40NsNJ552 zC`^Q&n`|VPTRMm?8-@_^XdD0&b6&7d-TBk=dOI72j-7M4-ko!R{cM<`%v2253P&vXWSE>8Ald?}HeJGaVc2x8d-KHdHDk)J zAyc$w4B5jVMW=pdKdL-$0w{awuKQF%=&10<_-H?vH-fQCrJk=K-|96Z*%l)imPXpi z_=MS1JRsW3rx4SLUL}Pw9k}iY)xRN&^nLp@0s>`(l!rbgO(oAH{kt7V#e=(v2*hNXPt$3X*|2oFwZ>Ui{#!OI<(7Z<$`=yI%*LS{dAnn`} z{rTReKs-R?p;50qCub+lHPi#@vj1|u=*f0`o^z3ld23Q$RfHKn&G>eQ6fbHu0j%xU z$jNUT;v&!o@a&`}!Ettm?%AP?%I&pHjai^PC_et>uKRU2tuLg-mNN|oVU z6fOxBFf49vt-WgM7F!_~e zZSoRx?NXocR(EuaS*WMeM}Q!6o*oUpNPxbB?Z!pSfS;&m&DGZ%3nZg3uPA>a5egX` zQ>- zL_hBl+y9Sy^k2}-|3Vl1+n~ErM_pA*5Gls&pRk{wnWt`3udjD+5PmWh(*AtqyX zDv!B_W#h@tZ-EAL=q~HQ-iJaUoccBg&^KMf(Vbb0zq+N z-7RhSO;BU-31=3%TJm)LgfrtL&7WE;>No|s)4I#jpXy`)y5GpqbDS)k4|kA8y^6(+ zYDsJs4jkp=5S|!I7Mfiuq{;7$Rg*)8zhvj)^b8C7aZW%$)}ZEwF!aFDDj<|mjMum# zWqEO;#X@E=)2=BK58jckjz^r?XGmEBTJPAVj%T*0<3B`0dp_oyW;CYFp4twA)pTPIuX9h|JH|Jw9k zG1Z{Ho)(>RCcTdJ!ZiPmGwj4}Ax`qKjZ3stQE?!V$_=yiy@Q( zMQUzY)mS2{@g8>QgrF77sm-Ch)P8QP8np!|Tjw6Gbm_M~Zku`8v-UR=GC7hMlNi6i zONa8Kt0JoI$ir)cvB$<2ay=T$Z;d|S7K<`Qa@$_1#FGqh1N|3 zpNMp&oo5}!=?1+Fj{P>klMIm&%PvUsRk0i^x()!$+QPc5Lv5tjno3H@JxuKeL>3yL z8U@9DLT4Owu7#(P0S^q)m?u>s0bf}v_5cAH*kdh4`a=EWwkUHS^Q_oi1IYjzWz>yX z`9Yn&-S#{F=Fto$q~f1~6oOSBhDcKntCHSfXV~xIHr!oTGQ;E(W?azNb?9|?=(B(m z2wm-N>F!mB(gVD04<)DijljJ$+50h&WyHIy-v(2&**H93GnsaTQ_Q@m_oB5@6drYT zC~$9Jzt>w}j^hHV$2Bldy7E?47EWg8E&Kiul}TOVNlO77SA~Iw+QIRKN)I0;8}Y^-Vh0h$OD@c6T6GA98&j@ zr*?QK*h6`kMwr@pdPu8fyxXFt8?Wah?ZQ*!{EOzb<~H|}iUcx?3Idf;=Hl++siIiF zoD1)!9xP{ZDMvI1h-wkh?P{;@Rl zpDCRGUf%y+>;+9s?HvDsiu})xe*Du|0+rXoW|(h5(}jpQT%27HV!6sYQoW{WVK%DV zxdWamEz+1N_}PN~6^LK5W3Nx~(^q1j^vJ>K=5NDeTrd14o(A4C$OVG=4N>QE7N~}h zxv?NzjXQzr&A)6V-V|&zC4Xs{SXLa*r7lR6C?-JR{=~r6lR|mYD>?3`+eHo3ALm5tx0%a6#RZ3|TtHCp@In3KRR{c zGSNL*A(Cmyoh~nc`Vu0N=Wq#%bzzor8wNBl49>bfH@dGYOts|xZwI{KfEuK~j3rJ# z&;K50{l5{isD=AKICnY6f3y3d6m+Bn1yD0t&y#&D&{2_m{8yw>vb~U%Az4JK1#TNo zMpc(K)gL6^8}XmNy#4s49YF6dk1(EXa5!&h|4?z5)Lx<0 zT9X}~qE>;=xt4L?)F9%u9`CN=Ceju(T*as&MOQYA(fulNM|v>$_S$Pp$p_0~@btNnv>prQ*&xky2g>iEHB#BqiVmtz z2ol4VtlFe8C-;~6>#922?jBO4o2fiB(y2F^8aV0voUb~RO5c?RR7+T$o%W-pBdhn{N{=xg4%OAGh;K? zI7mDQEe=Cz0i%PckVm8`=w~=4mar64#<3bbx?tR7=B9Xnt)5 zxp%=pGrG6gXXk*ZleT>t1w(w+PtsUw_w~pg<1m=}mc*2K=RnOnBZWl=EcALUkOd!FNi;Y)@|B&f zb!eK6>R7Enl{%c7Km~Z!M2SvH%&TKQddN&EUq^i%T`M-}bc?3pSB&!;ck)#}XbZ30 zN+#!TC{+}s2gH<8IT7g&QVmIOR;p<2v1Z!IKdZTPr4qoUJe`uU7t52bB^O&C=gr}J!f_= z>DN^*k%H=Ey9pY&pD1UfSFN7t+o$`RyYA8T6e)4Y_2&X|tU18uB`4Odki07f^5K>j zuE4!5`!QncccfaZdYy1CsO~^g$r!w5^uVMSXluauQCe7Um&j_ogYf(=T_@Z8mO9n# z?&o%Zzp@J8basrzw{=VAw4G6Hksw#cS4~HRghC$n4Dq|2FvAPeagU*JIM$G@;OjlY zjO46vf?ww2!X+gz1kc&Z`6XQ8H(99AhzAeR=Js z3^}ub<^&>;oZcXrvwdMP44C>qtz8LRj9>R3SyGa%$d)!CT0}yMR!jRzo1~^i(@bVc zMV6uvS`aCeM5t_Sgiu6L*(#+_DkTbu?Emx3)HHW`o|*jq@ALW0TX}!;J?lO9+;h*p zk5}S?D{Hnjy-pgCn;iVdY?IP}gR0uL!+po!TiC4naAIPpgvaawzBPktA1zt=`rLqA z0mC8pXCG>_tADaDF0m%q(7%?ndF{53K?YLpQ|@o{e4m^&Bv&Y5Jaw^pP5o}uU0WaS z-Rj8LG;QqPW2;&^h76^}3^J=8q&Y>xN@Yoa_lb%@WFs@{lDXv8;gLDRy`|NC7&A^#1+u)B6&r5| zP2W3<>?>3bh@-&0jel&d?wcU0P^qJ>T`qb}|K8T6%Se)h*1Ia;r`lJmYs{q#0` z9{q8ALGxwmyEX;gMe(V#;{iZ1(ZW!?Sz+(inT|g_CGd1=PF7r7b7qEX+@aLmT=J7Pqd+H3 zt36@s=Iyzmc{EYjru=J{d6nk*$nusd7m07)@#pHIW*-0r7X33=J7gz zw+R;PFs;h2@h(%D|91I(@07d?YBf`DU%C_fY<2j@=H=dJm2{;wnq^dp>3$+#b;j~1LI^y<&Vrea&mj=hdZg(s%3g@_VOv?= zL%HRh(O-_rYzfy}{A;X*@5T{YyDLA1EpgmqaCc{tmfW+oa^dSVmUvs-s!>;Ch%_qi zeQ$B=r1H<@t+BVlq{u$`LGGH2X&)ma@mzT?K{K{2DtqNV&S7IzGo{ot+Y*4r;) zhsS=~9`*FQ!YsFM)LPEpf zK)IFX(c?#E4jOHdx8v#X7IAhDZJ;oLgPT0>V_|0g!uD*$-z9RGR zjFNAF?_E2?123**y?nDW@%p7}D=LiX2Wrk8i7_y~7AR{t+1xww@W{bgDowi5XAX6? zcN`nIZP1LYIYsj4*(K%M=Y`WZG#R&kikW%x}oezxVC4{-5M4ix z#?5iw6)no9Z5@JIx27mJrINFktDDBl732l}hz+TJu{Xe2Qaz(_QOlKP(Rkrc_s(87 z)k=PF(8EpV`jJZO7qK7Cs`+&q9I~_iI!#*cuciMJtIL`mM`kuBzwCH5WaFH3QMIMH zM?|*VZ#{PCTy>yAS59h&<=Ir5$1~{eX)n(Bf42JIyS=usxQcO7Ej>;zK4;}DkC-XR zn%?Pu?g{O2nYXa83Lh9lfAG-`9LDF#SBaCH~-9>3L^U()?2dghQ>pZ_vuVxTH4(hKcAJ;*j z`J31?6Q_($mJ58gZ}3F3@yp^X;xD>C7abZLA$Z_e(2h)NuTv+7q&{7GWL)vV3PyT% zsJ!mk;Fwj62>)YVUe+$*!@gXmtx0{o)b9B7xf49CZ%Yfj5dFPW{NhE|jd8wqRO8TE zm(bB=f6AoPe=k)1achZ1V_@xjtC-d$N}}YXD-~B4ObFWLyZrsp5m^)|aU*A1oZmK~ z{J`;7&nOIe?Xv!U!s*dxuGEZdN`D$TB7Ro>#rK)Njo*6j3b#=|rT%lIpM>q&50=fk z1#5&0#~hn<(cF4vd#AsO>+Rn=7M*>5tVDF{yiR@b4~@SiU4wVr8z%GZ(T4N8?|Xc? z6#VAxsvydAu`h}#cOs8Quc3PUT6MTZZ(JjC#QkBlm6BJWSxJPYaMIvIp_xfT zjiNFxZ6Enk<5fw!YrF5Sc<&SDt!9$c4QEWtc6#0P6MgWbqoJy(P(nnr<TGH2tfQp{&l+u_ z|Fo_cblR|K?DE!~LGw0GPg)pq`L$@wH{&(u{+y4mSye2WJn-fofuszf7c=JFxR@v~ z_k9%)xzOBxzIMmx_-s*f)k;UPpwxGa6aGG?K7M9{B!1o~+O7O}+A^Bz>&E%?jVgn+ zvy+wwm8pUOiaNll;hG)c7kQ!8v|eu@7P!a*yqNwJdS*y?r8T%hPXYEo|F4m+oK^SSv23 zWid+ge5SjcM7i~^!6Oe^Q5{~%Zah9x>s8Uqh?NCvqfYvh?J^G}I#1v0IV^M1^JPVv zVIj+n;QK3fELwQE;^ZjH7xCFLhlcL8s&SlbS~BYNZ`;BWr?fFjDjR-$JCiK9WRmQL zb5_m5E=B%BQ?H+*3kp&$4@Qs77#YRjl8~h=zuR zqAN_E9gR9rHC??;WTje-qUh$GKZVW*CYj6?9bBn2zjd$K(e>+I2Q`uJ8#%PSZ*`N< zGJCy8b+Ed|!|}@(rfEF?C9ZGt_OaAv(r4*et=_2;GFLiD_0xS_jnb{G2mgs$)n!z= zWR>~5V+&$b7guE~Iezm^8l-pBPjecjEM4=&opHX)T#g&+pR&K#b>_6J%~Es6`uPEm z1Ky+;hWX7nxy)uSrE=7(QJwmcv{&Us^;#i>a zen!H+w_1*LMx)LAxUNZ(0oId*KfOJB^n%vr6%`%6*%EuJmU+VaB8Okjm0R*-&rY#q z!?}4shLVB$uvzUuTWdIgECg%=hDgB zk?Ag*gd!K5kSTFq_;Ev;y`Z=5!!N5;o9oU__PEmJBRYGM&F^UsP0rua&)YP{R{i+r zn$WKY7}GXAGifh!6^+=CoA%mK=VZ!Nhl~;VXPtCyy_VT(7A#KF^(5s^-S%ynvV-Ww zk9GcmCHlz^3}|DoM{Q6)pGMB~wcR8$%rAS|o0$&xj87y)Yg@HFad8M9q44mQ$c)e8 zhc;i^9an99R&3yD+8dv3sfC6v-uLh1{l0O^c5y+f@7#RZ!k+>eG^M|x&iX28UXxqB zosVx;mw2by8u@8^(UsjIH-uKvbM+}>i_3GU!F2IO17$Od-lW^dY<5UDI-&PIQDCIi zpJ2a_ql=v{C>-AQdD@CGi7O>vglJa{T$x-@7cef|YwyIejouk^m253aUze^~Cao0{ zE2~F)x3<1jbaa?bW~zo$RzkrDQNdPuiDbnJNmt#+`z#{HSe;6c4xh6-<8kT=vvGCd zb95C)*k8}p+yA&m#_HMhIW4ZYz9t4vkkKount5FC*J@haqZLCUoOZXE&v2DsC}@2+ zwB6^VyjpROhS+^~(aC@|Ake6W1SUYoEEg zt~qyH1D&Rj@Z;T`dyns!hEjz$G#UArDKFG{aLas$Dph#)bbXocTT;yOX4PcOFg2bT zD_dT5C{oT)8yTl zYIVb`14S1iTjd^%bs`_4H0&(j*dg__vH7$2Xs1`hsUubzF0c<*mpbT1Yf#RZRU-e{ zFgI|Orgwgz=l;k0UI%4e_h0Ehx8?Mnf}5uv{M>DHaiOiC!9|PY1j%tTcBMS{Uf-hS zxaIwFr6;B>ZU$EJM)gaVQl6B^9NE>l?mPTHtF}js3ky#Xd>{9FztWSEmDS&+YiG6i zPG3^m@P=|@`CHRHRd!lh=~GtP*2^0{_NcrfZMeQg^Rka|fr;fr$~l!Sr2PZ8Ow{cd zrErwA-AY%JA+XtdRUP%^uem!Xs3$+6H4M!z9j>wFaKx`)yQ-5^c3;^%r%F9=e&hio z=M_)C`cwaiP-D{LFHY9f$n{*O@YkX2mD3TkIMoIILk~XJ^}F@kq+sg6Gl>f*<#Lbe z7a2(Vyx-Jh*{CsmTzOW!ky=TM#QwB`pfGpoKMPOg`{%`;Qki;WWAsM*KZ56Pd=Y(K zt0!w!(cs`~`XRiveBptr0?MVR5gBDd8ms-27~_K&2_XW_x6fX?*)()?SJhgX%lCE& zZ@BXFf>6z->^}=XNybmzx;0$>QdisuQ88k$hjCjZ&FbLw}I zKW{TqM6~DF|2Xq&h4wYOt&^JtqrXI5yz*$!1NDI^A#uCD%dZ~tQtGR`8~I3(L-Ld{ zdB=BF>@Ge$*RZXDa?CALs?p`{;qj$G)jRCB`gAHw4*KZaHMBf2wP^pg5r5|GrQMSn zUuxKuC2+87fo-u^YTM!zIp4&b;>&XfbqXHQESVBwQm6J;J?6ERZP{3zz5A_j)Q#^&dWG#Ur1|j7srG zf8I?$ZlUr0W!1IaGmoAh+jVK9M3csp;-IR(DgI|dk9j-%NUXN-6KQLfwcPtL$TcWf$9jq{Z)k{Z8zgfXj$qjg=!rK8+jEZv8fC>Y;g7%K|H}PuLUDH|qk2hW!ytj&8aVnz7YMOdVc1qLW8w*PEJccY8Xi+d#^YTs8zl%aAE_;|GX7Fl| z^tmY_#xwof+H7w;3$rtsAmnoMJY7(Kpvs3Ke$^+23{QF|tngx;d4)!$=XXKNpK-Z5 z`%6na91JTCJg^&n___IqQ}G{%6c`VvyZx);R=)WcTb(tPtxNAe@mNgFyH zwqD0)CKwe~lGBn^o?LH=pEX@**#fa^(@eH&lrKA+FQ{YMOmSEd79@T%AeL6>ewpG{R71om-S1eWklmERoeRuRta^$HSwT?$e z#9eEQcq~*W?iAKo>}mTVTT9Y!cHZc>U*#Q66s`)ZdHnQ1!JWX(%3_bUeXeUdcWBdT z^~u(YhbLD{FWFf5)FWt>&UdHQCVPo>H$MwVzce&mudT+YT-HSoE1e|Mp&_b%#dWd% zWl2G!H^q}Aw}#2Io%SJrXv>{hcw&#uJ|D?e$-_UcUQJ0fQ}djX)%Gi6ZR>#-ftrjv ze>Wu6N6>RW>dsPMD|P;d!geK>KSEeh!t~)RI~#Zzh7WX`ni&od5E}?RT(Cd*nX4=T zHH+?NB>bqKN#VufXTX2gx7cA*eu6)mb1U(OAhe$^Xz7rB_}sI zPzA?Vec4}i>v{Dj&u^oIli1&8-G9RKt61HK^gueuR~pl)7opyp#m;b9_S*w#j<$YvJ|1aXnunVT>9e-y;Y3z#Av9MsH^dPvTMt$R=VCUk@^k z?n&k?yaIxDwI65p(4uE}uUKJstvbpr6`jY8XKEhAzL1k)i z88NK4a{y2NrXWDU#6Wa11FkILEi4L_#{;bUG9O-=lR-H9Net!!2Ls&sTevrlV3yDT zZ|}7P^Q1fWMiZX8LdTakg3yoQiEYiZ9*lvbcwhH+rh5ALQJ60%<;e<)^&Afi&Xah$ zd4h6a;Pb>9)58|im&D*51HFprmorX60QB(z*0SCo$lHNXv9{tPfs&lvnMXl*iiL7@ z4Ihh6UhfBQzUR#qI@mf1kow`$SgQc*!8||-<|Q0W_gr+v=sEDhn>pzoNT&7ZyfImq zXYm?!DBLC9aCEqIj5l|v5kBI>v3n+XF@}1nf*_~v-2A96eBB(KmhBM${(`_YZQT!M zy4_sgW?fCji+!n{uQSbb75Do-&RB*3(I0D&;_2HG2--JXH2$_mRjLRnTdzBE*@Tsv zVVRC@gIPf|O#DKygFTte29}zw1^`xK)tw|^SBF?IhXt$ug{7Zfs4CG7uA1?=0QhE0=RZ> znuYN>Mzb$2#;2v9r!N86*l9<0&Hx)Ng>l@QpkiQ$4EuuhYLfYz3#3W-y3BbbE`0~| zxBzlBRF`AT`@;4*QqQZbxpkRC*|9hr@X|n^AU69+4py+l$79M&1b5kUzxniGFrp!B z^(Gpa)B~;Y;V`4%i+XKQ4+wj9D`-D zPiG1wW}fa;tlUL30_=<%T#G?Es}=*BfggWKZ7z2eLXe8injk7zBDFL?W#~gmNhT2SV)diY?OF%RU|p zzKK8I9oGiBL&0Bc5V~Hl@aVsG2?ZBDY&tmB&Yu)pTlnCnTKf;X!N3!PN(3s0!zcdW_;Pfgdgr!$i1GNV zE+EHhkV6HLL-8wrQfxU0W`SLf$L4B6^jHB?vl4>d_KiP!k0AI;8l=E*90wEfUs4%JvnV-r>^V?MD|{3YE%6OW8BC0=BvCpWWqH!+3ELKAXL!MNpZ6+o zkJ}Fte$M(ZX3v4q6%a*@n#z+-w}TQnzSFT!4}?KaiC_dZ1o!+jLfoDmR#vUM=jf&*qEHr6YoIU@YTEdk7?6fV<8?%rHjnA4%ZLAmA-)WdL@@N^GbRmcV+K zU_21mc@92_t&bMRJp>ZM5R=fPqD6x*3Fd|gcBddqU-%TZ$o@%D1`3`a1R99OEg`1R zz1^u#b)nE0WLJD>`;&>XD- z%Ykt}SF$dtfxAmJMZYhj2*592120!={#7Um$+jUNd4qo*rzz^i|6t8D^GGJ?XR zD&na=rvE?&x?J$ZP|&$nYzV=0F=$K&!Ixb5Z}@Vk;s_SVy9Oud(t%bes7@cDSGB4? z^iU!oRu5T*HkE)R_0+YvZ16Q3sA!?-UY>n_NcLFDz>@+g4eZ5W(>cC2&Btj~D1tUe zfrMyaeC6~nr0~%dqn2ji>u#NMf6!r-f_k=G_rDP9?g;tGA2|@F?_e@@{ZAzEB};I8 zd&v^=!S%3Vsf~C=HCznZ@4_UT4HE2ECaeP6+KO@)z-k~k@OJ%P`^s@#3e87f z-swe4ux+TVpfA*+PVMhML408tH@xW>>sB{8UbI??KNE#XaKn! zflJEbgzMgGXKvyHG_**%8hKK&>$Mo&xj&``S+Xake!bb)^Jx_A*b?83n z9bDZyVd1Uqy;+?bFE=u23Iuf0Ko~z#Jz(%NIr!jR96aV#TEs*DN}a{pabPX1Jvb|2(=E>gl(_-hm5a_->dii%m!T?1v8?#uEZODxNMKsrNPmU z0D?KR>^9}l1@P-pKn0!QRiH}Bd)%|MP3+i@Y2O|}>70}VAGF-6(Zmnigoe%g3@Cd$ zds6v~d*zU**Xf{nd8l%rcFlOpk4}%4@b&-f@XV>$Hvd`BKZ=|Sch&ON4r^!<`J%7< zicekOG1(BSP~^Ig{E)l7oS?iaA~bFOL$GKR@QwP+oG<*4d$b|PbA7a;aYY2S4{#Fn zEd`W%p5$QXP8!V}j{0Yw_$Mh-vF&0StVYtuk z;Sqi&2et1j6&0rX!e;h`M{x&i`x__hp$wkA4_if#4- zDc(Ce3gdJJU>$XVedHJ>$d3@G91HN}IPp#M$yk6t3m8w)asC8TK{$uXlp| z!Pi6P-MkBm@d3r4qw6LVwfYY~58tex+YBl}!6a4*5$M zl%vobMo0?}ig~<8&JvcPuxkY~p8<{Tjqe75&y!zPgO#8#O_F0<3!A#<>){|HO2$Xr z;<3XQmYbwP$mFCEL|53U?;|!=M`5<05X6V#j7hWFL3CMgD5yWMpXA`pOL5>VSm#0U zSqt9vk|GPU$$AI}=+gd*F%A}wVhle1ajV2UC^V5~g6QbtL}MimKB7!!Zx_MlpY5&r zS0{mY76R|`Yz<&W(zC_E?XKqV4viNCk_`gEGmpYxMb9H;Livl(O!$H?kUsGj`XWe> z#&8*F;~E?=aCPGPi@4~=QP_l#2m+)2^T8Phi|s!IS%mCudm0u+r~oO2=+4{~2d>+H zh!zter71Q|fP!uBh$Cq2WKJ|~H^O3pPH?Fowp;%V6h9S#kbp?nt9W*kF{?|ISN$RG ziKPaXzhJwkXks6?4i^$KJDNYXoMgx`U&|*)khw?=3$nx3AAcb3puE1L-~+?*m$%<- z&rp5z7(hgTI}d`NCy2@I_sqTj64eb1|8gX8*qV_Q^^6aNsKv=GaWl;s7r?f zcMvl1eO!1+WL+uPO9YrlBV}g*EH1fORMzpX2k>_B+DxBRv9*pzBk=t-SG>Cxlpj$CHA%tiJ~Z^l=)fptU;n5Z)9_ z{3sNj(5hYWw)2M8rcm_AZq9Jog&7SFXRP(4c>2O5&6C=txLvuq zAczlmC0b~!*u|R~YcGMfAqHmEhJ*w9H&7FrhPH+CMrZoaT%N2S@_J@G5r8LvvQY5y zz5M{6!xQ{utY{c4wn+AUT*x?*H+Z*Rm>YP!Qk!GJR*4yqqU(;}j$JpRaVhk7ypDQk zZwz~}q!YltEaE^#M{t3m<&eT_xH3ee)gYXb#=(%4LIC6AaRHgK6E*ZO=w2Zyz@8R( z9J(E5c^nt5C!529-}Ozv4vqlpHS-*Cdc!6RTC?r|3AJb@27 zItxzmqsJ;@u!!J?$@~71ha5mnHXspdiDHOrI1@65GzQLIs2unt3?mtqGXTC0hLJ9! zu-#|)kuYZKuDety*R42S8k33JzabY0L3=sZ;!zeF0C>xU*acsneZsF2;y`6P`Z({R zG=A`XKBpC{QHoP^3S)V_J z@RN7n%Yh{BkVt7G#y8F6M*&|Fo`YqPx7~Ei*DAo*bP#g&IsC}=u}yakke92KEqyrE z8pI30f;d8NQ|`ah>-KS81n0VnD&7JC^C9}6uUAzdEg}Z2n>#>DM6DR{s`!k_%gyXs# z{ISx0g3^ zfu3CCBH4Yx($n4N7MR>6TMc-pd~MneF6I`EE~<`ZnY@&d;e zsQe7?KE(WY7NAE#C`9)_t;>0%a~d9h!f>*WmBv;(cR?*n5wp8J??1L2%k2GK0A=Q# zwugm3X$Pol8bVa#1ujv}#XOM%8yTO;90GuiF!iCO+0I&AQ0{S!ub|Cit3?_?IZ3dX zM`t>VdR$oO=JNAq_VL(w(%?=zzExC7ximQ=V0$r>mn zjE2s2bV`eXwlN&<=`k#81AH)WhDb)&ATQomkXuYRUuvH{82f%Kwz$aW5rGIE4jOGDQbw<}k zAK<_#50!BchAfL&3MI&3%6x}@v>gteky6};bHXR}mjogkz9ft-Gp=TX>V#o%pc&%| zh5tl`C~~`2Q%kTe7H_Z@dQXZc>&49OFCoM{ay*MoS5#dUu~NMPgc&qCUR3)Ra`;+1 zky0uFVUQFE^T#|ynAPh2MF)3gz4n&CpKZR(8ejufX&K4A0d##KPa5nFeth14mcMRG z1D*$hcF>`?PKPHhGkVRL&k??eTsPy%vTT1)Pg$S%9;nNo#9U4a1T)Ba(IIc1fj32f zH=z@_1suWTF*sNkVd}%hf|#0kZ_3Y`-zB~qwDcH6K~EL#S$QVO$+S7*u7Y2G#;Hn)y!!XQ$mTL_V@&L~=Lp*nz2~T3ITT!u> zv$Jjxkl1eOZGem>ca@MPLh>8?9^ zU>r3uZfJ>Ep3t1@V|=rSSR6`+(`_Vw@G5k1pL2pIbf48RQF^&c*H*~^3||HUptb4; zCwWr9k`rEwA&L3%opb>H39|`WvDtP87npr9x>cYL8S4%1t_|U9WTDTmoMGfh0MIf1Dum5p;O^nh4(=HbWjkQ9pAacn-IJDr|ay;;4RG# z=fGiDU*1U|f@a4bv-(NIn9N|nUM^2r{xl70H-KgR9VoWQB4K25E-o3i2n6e@s+90S z>JV!!!4lIDz)a8?juluQ(|s_6U_Iisq4_5q>n9xpV05T1DaHluEmZb&3K4mG+`@U(uwr6{mtd0poRPJnSp5IN(m=A`>hI0^!jBLbcHdR^Du_s#x16?|Iu1@N2u`bu zi2IihAcw&)V%Wi9IXzD|H~j!?ToPV`K3})};xe$@ELaMou}ZX@Q-Xi4kB6UIKHx1# z;13!`&nI7~cPXaZp?{V_Vr0;4v5mjXa`A<1zdYiMZT2_olKl z80Qi=4_el&6UV`1maI88A-i)Bo~o7Bcq$LO;-v9F0v(X2#`h%w6C~!M1X)O-WYGQ7 zAo?(HPbowNx1c+pTdCbyH^AZhk>*{qj5DB2Nr;i?$jO}C7mXc&@xjWU4tTg9G-&{C zE{m{sR;Di)+k=R2{+t|=b0-8U3j=$ZA1)Ml#SPnQPn z0>H952%0i%M{=X}aXq*!4cpq{a7L^k2A8@L?Q@3N-jR3@bg*XOi3q)0i#X}AvN9ei z)u`}8it_>FJ$zt|&VG`EJ2g3hh@JYwgJbLAk_%EEC=v~_w8fkV^zN<$I5Y%eVQD># zWP)W*kYM3sKN!gmq2~adul8%>VKV1vxX1uzX`=Kl{Ne8laElXZ3b46^EhgqacFXV} znP;1N-isy&Cz#p)z+myt^ANmm_{+%^Kn3%DO@x}+3Os6XnuGH#Y(0PVhm@+@7|$*s zqYSQx*7Twc`IFMY(vUtC5$vQ>ue?M~e|-6k8QA61S7At^7oZO``5#eUPF;sJUel;p z;RAmlf7#zr0i8OeV3;j!5mmo6`yX)u24^l!m?|NdXfL!I4{8NA!hj7sgbf>bcNk{` zV7<2wwRpeG0Bo?vGcP`(%+v=Q0C&L7Rhc7v6s`Q<@WIpZe>6^cNd-^LfrT&HP4L2& zKY8XPkN6(G%{6Zv(MyMxFhyDJ*D73C&7Vqtl#NgKj!jtHL8vo7fH5tHDEAq>nYFKx z!BFg2hmV-iI(d2a7!s)qwldJ}!rV|iM10=yv9DfQK3yAJ0h=}z5N;nH!NX>~V~P2{ z%o&3yryL&;ITGtJRscJr5uY57hs&u2qMM~tS3MZp4vGI;uqay1O+H3|*&8si(*+Rt z89Y6+mXNH5p zurux`=J+hWm~-&?tlv4}v>u4=0EsbL+SmvckG^ipbOC%QQpqKTDu_H5SVT)OLV0*l z%uvOiM%jPigO?YDp2JSGD8Ph*${u@@0K8Y91Xn>cj`FX@&TftX8=(yAz~TA6vLmSk zI2MJhPxU3blb6~cmIw9>_pbZ?0rv&p=+m^DSp8!c6`~n>g^ddYo&gwJ)WZ;IUl~_K2y%`!^e`PM-2~^txAGG#;;u!%Ej_A{y z8{ivGQ!#YqT@dvW0=5JUI7Tls!!(}_s{k0oemEL4SaAAKE>WmOK&eLZNdL=)G{ zmMJP-pb|HjUuPmNlM1aGMBuD|$Fm%^>hcda3*dTkA2a@HCkF0y7J>n!*OusvU9ukm zS%((F@2StkuqHI-%b7e~umuTwp!Qy$i*MJpMb|aeK!kUocr=JHx`=7@ioopu;^*$G z^IwEDjD`c7&;@~pnc+YCK?I+3?Xv9TZ7?iCB<^aNH z_w^c;wNzkr2e=|yq_j}PLuM6A@UgB95`AL;Sn~i2os>nC@vztdgqaVr|02k*I#Xts z!o`v#Y0v|Teo+MvonXo*h=(iHiX{$!C(D7eEdulPJjuboRQ~~;mDKUvmI)2HI10&* z4ip>Ydo%(+lY^TUaAL7LXqev7y~#sh#^{_0;u2uOa43*(j{<{BKL=j-;__snS@Z(Y5MyxzK6=gyqp9At3C_`%a6JuhD!Pv48Bt5~=- zGJdG3sV|$Zre*P5T=a>!>NJL_S)Yo2s--=9Y4u!1Ogvh%V#%{7Puic0KV9;ov88fn z#ZIuxZY!%bM*!`O1!9EDemkRjOB5*Uk(!f}AD&;7lcMjNn3<=SoRe5woEzZleb_;u z_VyCb$!^in!S>u$0*ly-zXW_#>SzyZ?U1v{y|pywrizg8(fLc+S^57l)K9UHZdEGb zH@42df2MkK{C)ZP41E{M6Bj;R&^wjQN^e)h)`z!W7)Bg7ntGJ8S@6w$gBiv)%d@k6837p&!nCF`FoeDuXJ<;J{{#cR$Sf8>+3$yn*r!e^Bm zE^gTPEKydP=WOx9kZqr*Jk3fIWw?I+rt^Nx1*oYhs(c&MlVRD=w$M3}(LkPSYyntETa(ug&Ef z#CKgdC$g5yq~lV{clW!iqZpryZteTXR_s^)lJjbaZTZTe$bXEW_}{hZ_5o!^28IpF diff --git a/dev/test-classes.txt b/dev/test-classes.txt index 2dc6b290ad4f9..e69de29bb2d1d 100644 --- a/dev/test-classes.txt +++ b/dev/test-classes.txt @@ -1,7 +0,0 @@ -sql/core/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/IntSumUdf.class -sql/core/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/Hello.class -sql/core/src/test/resources/artifact-tests/HelloWithPackage.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFile.class -sql/connect/common/src/test/resources/artifact-tests/smallClassFileDup.class diff --git a/dev/test-jars.txt b/dev/test-jars.txt index bd8fc93bc9f0f..e69de29bb2d1d 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -1,17 +0,0 @@ -core/src/test/resources/TestHelloV2_2.13.jar -core/src/test/resources/TestHelloV3_2.13.jar -core/src/test/resources/TestUDTF.jar -data/artifact-tests/junitLargeJar.jar -data/artifact-tests/smallJar.jar -sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar -sql/connect/client/jvm/src/test/resources/udf2.13.jar -sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar -sql/connect/common/src/test/resources/artifact-tests/smallJar.jar -sql/core/src/test/resources/SPARK-33084.jar -sql/core/src/test/resources/artifact-tests/udf_noA.jar -sql/hive-thriftserver/src/test/resources/TestUDTF.jar -sql/hive/src/test/noclasspath/hive-test-udfs.jar -sql/hive/src/test/resources/SPARK-21101-1.0.jar -sql/hive/src/test/resources/TestUDTF.jar -sql/hive/src/test/resources/data/files/TestSerDe.jar -sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar diff --git a/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar b/sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar deleted file mode 100644 index 6dee8fcd9c95702fb7909c909c6faf22fe93766d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4118 zcmZ{n2Q*w;7sp2#MvYFikeLa>Ac$xqA$s&Kf*^>_h{Pa?&Ql_K8AOZTTlC($L?_Bb z@4ZC(czN$7dFy@q-n-Vl>;C`y>~q#SXYHTzBP?tR00;yE;QTSC3HT-O0N4NpNp&%p ztfCaxc_#pX1yFv33%XKZ{v}iXzh;U{#Xn{RF-2J^Ni}s&1*u(y?k@R9FwOy@N3h$y z-F?NX+=IL`^9W8k1rAw-0mm$CJng0H&2%U_w&wzx5eux?T2xV~3skGDP7Bag^Bu%C z0^`pzG(1^6BrgZKdalaA2LLWhJN5p34%nA}jcu*Dekx3Sepe9wr7&?aw6?P{`BV90 zJs$i^N%JqIjER+%Eriq9%Fw~VTU}3yT#N$#MKzK$H~c0OCZ=j3@EL~6pynxnP~kBX zm2B)0PdF*nr>dTp*)7v^SkH~fQAZZ&V4kU%ywoI#iM6%Ke1}cA!>c!E=jT}U`25$= z&~z(5AucTrLo2*41Q{*O!!@tBGd?ZrVB^IZiEt4jF`rdAOCc$-@>`#}l}_u)XvGO@ zZof==jHWW{-w~#>MWFEf+Z}Jo@a{d6i!?_~(LUsk88=(%WxeA8a(2Wf2};DBP>G+lz`OL5BUc2zq4$M6U$O-a6^jYWRHXNb(G;`kIp{9p-sv^$A)_vabc{R3f6w&s zB}*nU{E8mRGH3@gWIN@g5U9lZI4-{5*`Jr!wHoQfZTuR>E9)8Wz&awzVio$@goCTv zc+oDZs+vP&_A|XC0V7vU->&~A8E3RK|Dgrv`o4#n!y|MA8_{c^j>~Xb1*OhO{6q3o zKJ8@Yb<7}cI2d8EJh76dABOgW#zY9XE$1e@Q7&S1r7sp&-_j>cK0;L z`^`)&4|P-SqBn8XdC>TtkdAiPZK#ZG%^xA?C2cHYvnUb|e8Zyod2stzlGX^l^bneK zKeLi+xH1f2$w2u}++t!p9l9Q&3<}+6UI##4GX+S1LO0c7tzklfH%=f9@j!?Ex*) z+^#Pl789lRU69i6czQZza?lPh!xcx2Wg6M0h6>pb(Hx5P{s;PgYngMDnQ)(GC@_t*flC zj-sOMs2pE7KImoyN0K^jSrL@w1zz}p zpUO3deo~dtxf1a<3N}bpIUJnLiE5_;er=MtxXU)qoG$AkqjKOK9-Ry);O{hrqy2U9 zLvkQDG$@d&RIVz`*G46sTM^=Q@!zuFmSGb|lyuY@YjF?!>nmcfo+Z-PW4U#7#5 z*Nm^r{^6I%)f)~UR7%ooF_pI;Z21;NKXQw1+7a1c&Nfw%EZU%d9n?L5+ZcrM1 zz!-z4h&YoSh-%afd#Bt5yPC(Hj;+y$tG+ZK|IS#NWJNC2pkvT&IAjp9J zP!d{Akpan$I&L;dyLCrC3KoIM0z)01^21A3TnM8GzZ5 z>oJ*)CAR5rF+l^x7*wSnJ95vM=r$EX2AFSWuJbC{3Tp4!&H1Oww*i!;+xaX(cLNTzO#DQ~m2Yk0i> ztG&&vro2LJ->}ctJULdZ45FWCzAlqkjWBdsdIKUk5&m_A=e?lO+micR+`%;)wN?+U z6{IXp>^2H==p>(1o!8ZD3`=!U?$OkuYP4g+f+A0jf&p-%X~>Lq{k4fi%9!46xORu4 zPod8}RS=6_KhFBiquDHiJh!=eeOwF!@-poz%lyfys16B~Q+Xmmp1s#9Hi!0nS9>m3 zti3xSv(1LoOw)?A2~R#{i4js0Hq!>E{5~aHXgzoQ3TxiataH5ZG5K9+>pK{u&y!zUCq7C)9v{cIut53Z*VjlElU^;@nbqv5y<)Csv_a z)vL*f+DuV=@>Brdjo({7z|VcRmx zc&jv-kZ99?B4%X*(@);M%>)$xX7&u!Mt8%*g_W58*+s?)Fn=x0z+~E z5ojJ_t1ZBQL+f%;2wP++d_DTX1hOEOLDB`JMjY*BKDjA^R@H&6PC)5gy>c9z+RZuJ z#XB*ApZd6^w;sfwUbIz=?7BbLYK(MMcfphQ0}__4K^aX&IsPaBb!EIdUQ?ckx!>0b z%qDS6Y@)?DDSI3zl9hqXvjWF2;iQ@HR@4L)j@j{-ek=8yAWm?&SGp2NBr1j3H}rY8 zq%dV*bGvBKqslJ0t@j-PLd0Kvw51rfHG0RGOl@0m>B+`xksiI8c3Z?apHVrJK|Cwd z^RWyYu?LgR)0;NkaX4k10$yNFV#WlGZ=Fv&bDD5iH11@($562l7XU(we2}eAHJ#}S z1~!4!a}|g9A#XloO0tu__G1y9nv5Pz ze8KhdvicyE^4HucjKi((pecyzT=?KIQ-OAnzeY{!w;WXtS`pKro4!0?s9HDJD6QNw zQFSx#llCmOD->gSe0zD`1T%k1UmsGi@7^B3u6#r&O}fW4I(~ueetwU{HRv_whi_jr zN>kpll-+N-(e_=i%n5kN?A6ZBK`SVaIg4#2>9q{1b*J03AI~DcOI6*5Bq5ij=ywGY z(GwBy#@^>;Bl|Ms^W2u{v{aC(-?(U=*q+{=dia70M;|r##$74fJoSEZ-_tGWJ(q!n zyv-fPxjyc_eOxyZePR?x?Xh71xnpVCwi9dFyE=~o#v`{88Ti|RH> z3Qm8SnU~}~wllrHcj#>^1ui{zzFGztm$nR^gmv}$3`d=p)NVet&aCI0P|wI$XVUmG zSWl|OPE|c}jBDirurS0rD#cww^Hh6xTb@f*6V-Z;jojMe>Jdo4@XBsBwHz&?ttJO)6dDutYEJKo1-yQWDxwYTcqs_4b z-ru@#Te|kVt)WivA>D=T>|202r)}EhJ$8()$!q=4)zvpLa^n=o=euy7JL+t@@p@#t z(#5E{zXfu8X#4g44%$**pJR)iT*k}5cvV6R_dE_tv48J*y)M3xy|^ho3fRXXV`_LO zfkoY z{Tnr}X!Q5^2QRPK^xxKB@$ia8e+K;W^&jgmWO|hWAMpP%@@IzT%hw-J`P1=VSozbj z@R#E!Wd5A!6=we0PG4c>&mh70G1lMCU!n8YIbUV@bt$h}@1G%lxpr3-|CZ%PBd+`i Sh;t>xxx8Ln4i-p!_4Pme8xa%$ diff --git a/sql/connect/client/jvm/src/test/resources/udf2.13.jar b/sql/connect/client/jvm/src/test/resources/udf2.13.jar deleted file mode 100644 index c89830f127c0c18690cec93ec229c6d3b50f7e95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5674 zcmbtY1xy^;wjG863JmTp#ic;e0#lsg?oj9q#Rr!nMT$GcrZ~kZ?$#p3VUQLtuEi;I zirc@cxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ diff --git a/sql/connect/common/src/test/resources/artifact-tests/Hello.class b/sql/connect/common/src/test/resources/artifact-tests/Hello.class deleted file mode 100644 index 56725764de20539b9c16799d2bff4ee3a4e7de97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5671 zcmbuDS9BCr9LB#%+awGS0!R}`BtWte0tyxaC=gc?L(vok3vQAzY}o9^&4y+#*n0tc zMQl`sB!VJ{6}yNPd-usFAAIom-PxHtGds(kIrxyBnLGFX?l-?F_kXhg-<`VwOu?TD zvgc~yaMTAy!IIjx;PPNaI2dWISkTm_HODIxEum;dofZoP!=cr|rm$97+u0lp2P*+?P8ks!ZHn1)p3q@LufjY`)nOUizAbA2GvJ?!`Taza7AxA;~q=|gURnSLIJW)TT zAS)7V*Ax`GBPht39*TtGGZhRjb)PP4RM7`&9x~8hL1w8q?u!BW$VMTt@ui_o1-@E+ zGDi0`N83B1ot&zqH2L}~%UmrAlvV|~#-I`fQ{AIngWMPu!!W!L@-RZd09Sve3PxJ$ zmueACW$cxx7@ZhW#Kw3uVV{yq%iInWkk&a5$*HG*l;M008yJE}Mi z#|zpN(rWCqfI$m5X`(qbjU82-jA??#2S5qfX;lWT%1IN=scGz};#ABMwArLp*=PaF z9w(7ksyC;mRhI=+oQ}DIW_K2_oHsSs#Gk&g@P2clPpgTQaVp0X}Cy`_-4x|zsmCDAf@v} z(lUevsohSpJUK||Jdt!UVuHkvmCRIQdEzS|J?YJd^Hif^1y%~uDm%&Y~J!~ggo*blfo=AEePYBYJHd3SIsnO<1 zZ$6wSuBK1p89{p1PO>~XNa;M0v*c9P}EK}zR|q?fQ$ zkaj6h8zRdh(G`)RhE6S(i&sP)(J!BM#j9Kj+dKKS7-H2uiS>APS18<~#e8@zlZDtY zF=9hL_Tdc{EqcXKl#92x&m~lgv?!R6s(^8==LWS>#XHz9aPNu|XN-vQzJh_pMe{?= zT2VY&)UIjqqE*qZ;#_>F;uz6oA3kO=s4v+H#(A2Ki(+PHJSMI{QL#5M(dS&-82-y7 zh*3}4>j|StLB{gQj?yrp$cJxK9Kfd{{ddj+Pmau*6K!Ed*1t9s(dKuxH)*kYQJ{+6 zQI@EUK^Cn%HW~}&Y4N4emW9C>OUAes<0mt(F4P(c#=Bx1lV2C@iZyG2kN{-Y&#J3m z#BH4@by3U^+5GeIj*~75M_zR?_j#+?{v(fObDP0!zQ2423j947xQ#~{_(`@mI7cGQlyrHX zccjbZ`M5wL&6IT6wB^WMHlsx%&6ISx)H~AUGK4mXG*i-L#5>aEGKvm~G*i-L+&j|c zvJ1;4(o9L0t2Z6F%S&*nM4Bn-@=EVWm&>bgwM3dJ>GC@7NSDj&af3vfDe3YS??{)+ zTXCC2nknh>F7KZW@sW0X-0sFb5^1KS%lo_|T`up(0}^Sbq{~OVBV8^Z#bXj_rliaD z-jObsPho>Znknh>IcwEUzr3bcd!ENei8NEv2leFUWqhQ z(&fwEkxrLkq!_8wKO>#sF9i52{*))f6R~qSAkS8-jg+<8NF216e+TtK<^aAnW}CsS zm@Us=z6alSy diff --git a/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar b/sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar deleted file mode 100755 index 6da55d8b8520dcc03c250a605151cc0d23a45518..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 384581 zcma&N1yr3&vNlX05Hz@L+}$C#ySrO(HtrJK-QC^Y-QC^YA-II#`N^F5?!7brncU0T z?8RC<)!kM7balN|^~y3wRHBs@ag3V!{DA=>=rZk$^A22Nad#R^=8KAuO@c9}#YvX5Gi7@R z;o^s(6l>$ot@z$juzE4)ZOsE}GR)JWk#%MOtQrbN8luJt7umofR9R{ zDoq9~;8-D9Om=6<%mk;F+Qg4npCf6*2Pg+Pxf={Y?EJP6i+1xO#$){Y>Jm@N1P{v3 zGejCuY=&RZ>`lTxrgm?%Qi+~E9}IW-BNK4=!r-}ErlH;KrBKPNB`CRxFFm{eSx)j3 zWO~cMc>B9<{2xF-erx}ikxu{3ND%J`Ge;|+!#^za_ZYbU!Z;YYI5-0T07dvOsIi@% zrIE9>o%tWo|2?psqm`AB-5;Qc|26RY5{#_uf!0>`e}I2{hg;j3`~mT<>OTQA;wk?R()NxvHWol5yFZOX@qZWxXyssJ zXZ1&Au>SKfmU=FKumawH7PK?6ceHT$)8hXm{13AHNtp0I&upOQU}W;Y?Z9uJ@S8q= zk;6&P0%)k`VEw=G^S>qjS32YV%LM;~Tk2T>Z5%E19Nx+O2ebaItd+F`&=~lx+@EEU z|7G6)!{%fEy@0+W(8BP~?EBYc{?}CRO6r*y{XRi|Q1L&^YG-8YXk`CK+Wae7|7+I2 zEc_?Rkp8`*|4-1~;hi_XSN*3k2>v{VrM02apS;HZB5vns@kh7uAB<+H=U`y^=dJoz zRv0^48T>Ca`t9cbC)@wLFr)@DJR<_*?1!&G!Fl zf`75uUtz<)Uz|Aa)BNQg{&^`Xy)#qcAD)TUz(UX7K0*!3MN1j=P1o2p?!-KnRC+bO zQ7U24p%AFyj}oVm%@7bzOd~z3?4Oj9WV|{emFzlKUtAoFk6#-i5#s@sE1>xmi5yMt z15IV~W$11vRcP+E*F}4Z)Z9{hmF{KRSj(@Ksg}6DDjp}uZaZnJp@wGKU|_S(5FEdY zO$Rj(QoqEy+T@C;qdRpl2oLQV>&Y_J7M4-7dZ6raGfD91%VY&(8;w~1TG7tubqX8j zk)934Hc_<=tbX!~xoh#2PTgZ(lDn+Lx~Y#0pWr`Uxd@rse+VD> zrWGh^#OWNx+bKAO@ zWL?StF1RpOv#Z7e{lvi0NtfC0lb?TRp6Jx^_GEmYRGOEvV4ky8=SIiE<+gek0IrAXW`PrpYF4d*CI*<{k3y*)e9Pel|d1_rbr28b6pQ#!7V zhcC2-iXCuYpNr2;7P`7V%J)inXcBW5^d#gu)|@(R;#cfKRtmu{jWIiFJPHa%$1%m# zNE@f`{}|F2;df7=&dAeCW#5s}LFX|acj*lUnwlNkbm`_gU4G?YA#5CXOZPD|Ir%t2 z`OO0flN~w=cTH-oH@|T{I6U0NuFV%3Qkiu{!W36{r@5UQ6FIz;9Bj0al)6yQv(J|L zdsx92F`fnlt?cxcmItli~(aT6SG_#4E{wl*lT~2aZK7Hkb{-A$A)v)IbYky5)AoSLq68 zMKtcVE*2)rX2RArzKL`)p#$(n28uqnWSZxU9F+9ugz1VJ@6u-O;z0sz0G) zK6HxRfxXBLMZvyc@a8OO?+$i9+l2UJ?V3P%Mxw*M2*kgU=*(YJfptM-3@`4Q#}6-F zV}W%ITr$VAh`_zzdPT29`|$3{LcE1{4PU9QtdT&=+^l@J;6g<1dtSPiuZZ^P+PUO- z)7vI#FV@p3_L4ourUC}3XwGC1Fk!+p6z$FQOsgMGM+3LB1m@SsA}ye#m#ONdL!5Mq z5r7J~iT)xG74@hRcDSJ?0{7WWvT((&Id6fv3fTiRwqUB;5@5f2#{g)J1+^h+se_?`_2kzD(lyB zW3B=8H417Q=l9ggnI>_B8@oF=P$88nr-F^=fB;2kF;%XZ=*S7-=9wGL*uIsJMBTmG355Y!DRCP3Ohl%+xCoJy z5wI2iRil_68f^ue->^9R%Z4N*HLRP?Dg}C{;zPY0a>{I7zoBfRhU~{g$~j}yF9xVH z#iY!qE&jNy&Si!-D$yq`nlODEh`E#xTVG91JH#fmQL$}+3^T4EqSL~q05v@*kK$$LUr{R0>CF59gpYP@+tRc@}& zg;N}$CcoQqOnwz-Q_S2~n{d<|d9u`MN-AHv6#;Kf9BDR5A54nyudbazK5#B#GclVs-y9+3@_?%4S=+7DY-Ce(lyfJSm`D%gB z@qfH#T^NA2ANK5!x?8*R1CI3Ui$wkej~-q7^KIxjhxZEm zyfq-8$-L|39E7EQ6kGI=y&sg8wCN`wZZL}ruPj{;wN~jqETT197nn;??fzMAe_cX) zEYxLh8_t_^oX7(m)kGz7YvMk=Go8aF+o9`&7s`V6C;F^5vO|O$ThV>!uedF!`_NQT zT9X@8$=s^BXsR=bs&R)L3LeSI_W&Uq8q04U45*aJn&@--MqCx-67Aus9jXCxf((5R z2$MG@8j zeKC?AfF*Y7Gr)k1W4#!TaC#h?S4Wo_@Uykq;^}Lq##3%i*`{up8{M#;?N4%*Vop6r z6GS*W1Vn(PxTC>m$)TD4ns1mv*okved61B<=sBXgOi4;~a!YCyM;0YKuCUggx$>6h z0f;)YYGY6Kqn-3L?q*?hT9$gP`6ebBpO9)Nuct|>fl`v;~y@E;Rx z`rd(`ta1&0GOAe>_F3yCuZJSrV)b1)1)kuPb2u~7U+9OlrGekzs-(MN;^r~Gc&vA1 z?a{b+{~El)Z{hOqb4k{+dzoPrk=b)WjG|=yJbmtwJchqsB4G*FcLiG*%KrjybGR2_ zmq5w0nNpU1rs(x%*Bi@Z+mzo^)*Mc5)bASLQy<59;Gzy*fsUsXkkHk}cC+L?=VgT- zk>A&DLmV_`c9n>&q0#K9#d9anYUld%lG;*U*W4VZgw)sZ9GD1s&-|6 zObX=AP6y>eTYEc>oa2bEf^2hOQ2gX8g>OD}l8lR|>~a(yxHsShZ$ zzmh_KAKT*pdu*%lkH>X6yMH`S{WH8(dGtat!TcptJ0kN1IWoW|ur}n&25~nU&9bZj z#RnwFe7QLAk94GI;ux(dA$4_p4HEEX4Y)rV=kw-atiyT*G%J1-i*__@{+#*MomH`Q zl(FJR53@04lIgMKocZ&4eTr%7^>U=>L-`O4)Ox6!Mm6S7=oUJDJBi^}flb=|4+NKD z+Zir$Lv8!NHf-C;K7jo^cBy*dCC0vtTX;r(0O29u7v$if9>R;65Dnle->Z)l^*=Y@ z;Hlh0=HMyWn^8&cF%aNE9V$8)HSu>>89s7Im(E7cN|Mw?n9+?tPeOWN>?BOqUASh! zi5j~}kE}X<2#mbtpgkr(X)fN2jU+>IL6T*$b~T0BCu_6*{vJdWIl6O3n(RKvU~erT zWN~b-bRGar@wm-!nV1^WS~lu%yw-nG+Z*$-HMTtNhUf8P%}lqCI9Yhr1sa`*y=A@v z^w-wKCHr3F*J|!jOh~Ee%w_gnW3e6WjXk?gR%+ql4c-vjvU3dXOk;*Iw>Wmt&&QLM z$qzjOEL`VvA6<%#=gQN8Vg-FLika*NIL;lMjOu0EpMF9(h2TzPt}4|KsRMIV9cu4k z9cP^6ElIY$fr_o1Ok<}JKz?)OBTgE7h56tDXJz;xuBjiUec#Kz6Teu7dM(%A3R4#K z$UUDg#wXmiXk&3S9Iri&M=Pe+aLqMon|vi^n@XE>2+VBE0D>2-%zO4aHnq}P(^c4~ z$c(iJ+iHw=*{clfKr_Qh(!zhn>;th358OZjnwze68xuB$ugyiC*q7w;*u(`Ys~#*=S3|U7pM$fQs85ZnJKk1>D=OP9J1Xze*4`Xi^rXg1 zVNriJzZ&}NKd<-2aIKVg(%ZaOCO+AGPQ(sKWsiu zQ#OlyxV{*jy*{k~*R+_xb>vVjSIq_0s?`8G;Ky8uyFnL`@o2>!r(W=M!{G~FTdUPVz7}L4v$uEr-;17eBh}C zpjiX3MlZ0|>evt3O43gs8WrpJH|oL{*G;WxR$UW>Cy<`r;(hS8gCMY*Aw2!@hOX0n zvbKvLJOi`plxS+^x(V7ByXp2HlI*avDBCz=xbrL5Tf5Ex6&(^iI4Ql=Zg~P{{HarJ|6km1_>6C++h4fMw`FtBv*2F9y;!bT zxnwIl+Qb!rU|y=DMZWoqGqJF5~V6Ho8Ga#P~q0rE&N&9QyWca(TyjQqfC2C~a!NC|J4sT9! zBe-Q2Sa_ahR%v904QA)1Sd^CZ*1Z@(dc5d(uVtkP+(8BASj?j^e-MU9Pz)6OQ7T5i z5K6j&GLF(Q<4%Z`U=llPy9bL=+DkTX(8#2eK{S)*gl~a+apO2rqv}1FF|0BqtwUEn zxq<+eq;-4pn0i33QcOawqR8uxm}JE58I3W=)CXwsyT zOA&Vmw*U|CN7matur36Jd>*BILUM}$CeXPc%Z>|Um>pM>?BgD|GY=WX7D z_FQJ&GPA_`1lrzc)V*BPgOIX`AD60O+E5?d!xdmnaM!vs5x9ax9NRirl?BS1s{Ba_ zCIohs5hbUETGJiCc?DFFO8o>>C>SUN5V8piGXPsMF6tcs1Bnn@LqVUl<|JnrH_2&9 z)0i&9T>r(=YFB9b0Je;J5N(NIL@nrm{`{sOZ)7EjhcO_{(iI)H|p_PS^{pUY&kkrIo=J9y$BCv zqy`kw4p3PIBSwhXtVj;8EPU*?4>EkZ`2k3caX3-G4ntqZ*~#wc_Ki+mI}a^OVyB zq+qQ5S+Lh_W-mor&fMH0dK)C`+1N>$SnD`aKQma6+9?0|Vc(!?QM`bU7lYzXn#TIj zl{Of$IFbRvLz*Qdf`K1`_a=_N|=yhj4?uz?;0=SVsYPdx+8D>OO3u0xxRQKPirv^2phX)Lf( z-K_#`FdIItow_nl2BcJ;*h&+GL%;`I-H^8kC;8v3@CiBKXk0E5%jV`vf%1exv-=pZ zRjyt_tz(9?Q-HRJ?Heo!c`|M0Mql=73hu)}2RWsu8Juae@XS$u)qpIlfr%%G3=}Wc zaJxYhZ}U5vu_1R#7PR4is=+@XNid|G%uxBMaQ)0yCnJCTyt0c|JhwY$CtLdL{Occ~ z$jWCYL+V$YJ$Qf80xVp$S@x-Czz{FxaW%PAgJG~UpmH~SH za$+L#p6p@-SVUk%V8~#Q^tB+`z1^Vub)c^Jsc^w7J{F`9pWoH`1j#rawUPX;MA!T8 ze^&E%$>-nITs_cCU;!veI&*IVol+m1Fnz1rJBmwQ7) zJ{n8+GZf`LqM*g9N$H0WoI;VbXrqtDRoXk+4R_064t~}ex^3B8YQYI+d%`miI};au zdWNl;{g%sOo}i9c1(dIqWW}b=!Ua z3{}bs%XF6Gx&u*45n1qslWBau(9@`=H#vX{nnS~w8KeBIh$L`Vb-@R}^=anJHIFEj zhb0X4$sCH?ZJaO!db~l+{gWQ^GKg{c?$P6#5R;#SSESCL-D?U zu(Ci;5Ci34vCb)!p3bR4F#;8x=JMt2hqV8pPsHh2DHrmM!KK4QTA_geTCXq_UM&4H zE3uRyB}Yl6zHWoonm=DK0>Z!>I>WYOa6hL%^8?1~KUloPqr#r>zVzllviNre;_q4f zmwFjVKnLU{)S+K~Mp0b2XrYLuAc&;#SY`fwFr{>rSzf@b+sIa8cXLNjzmc^6rt6Cd^!c zLoIi%aJt>-Eq5evx`XPp8FN>t?g^5*dqYdg?t*<9>*qtPkFOv5GLB|lFFa%MWUu!$ zUmm{jcJ2+$d5%BX`b^w-x-UhUH183*-Ei|}{@9BR9i!~5%FdaCx>Z>B)|HR zKPU|j6SOOC&&JAaTI@f7CK%Y@P)Xuw6>l|gs_hqS%fPmd(r^*%OcHQClNcNYV}Cp_)M++Sv2O@xBBY% zEiN8GBl~s?RSh|@2(?5J`DvtYb+Jp`6f1;WVI)#SEe5o{G}aQKycSp0XXob!v<0gC zkIgB=fPERUFW*)~2@&=1^f=Ykzw5EGk&i=&e=$txC>I%MPYfz=N)mQ%nm#e6Dmyb; zm?J9xwmeGdrY@YIzhRzQ31%7fHO(5PH~hx9~w`tYcGMFHGH6d zGAa>|mS8Vec5&p;6qX)=v)j^4{VvECgc)zP; zac$xooQ0@%$D!0}IHs@V3QmjR7aq&VekR*ho<_2`lJcIi)Zy9F8k=A@+yFU^EdIyntcmwYz!}m zmP~tbjPD95b&(mmZO+{-jl|m{4{ zU{r~%bPcWQIg!AC+O;!k-J)OmfTQX;l|X^&wLMy<#a>PnLKL5J(d>M*8)C~;@zphJ zJ>tu!*4TJfO;Cdti|yo?73}G2(HHTdtZQ@}Aoh%H_+IpmXrw{h0KJhDUr%le4h^3o z(s1y)x@nWGhdp}OYr>&3jCtM0A?=e(SR`_@;?UL=VB&tAY+YFyBuAs;XFNsoWQIX} zL7|lDQ~-;{T?A@)ZsDeq47QA!lvL>mOwt#6jcyk3JRA|H&`B->>b@ayYbv9JZ}VdN z;~B_YR#i+hH2e~G0RfF-gafC=>GxbJ_W^_z_29HQOVMM+b*Ae%D08%*;W*BlW5&@? zHCh!ZHs>FQu^lx=@AW4xkk#W@+QO(y-v5WL@vShNIYTO)2`&e>sA5C$%)o(ifT^As zg+}9kT|buN}tG<3)3IZ7kT^VM!NxE@eUUnijbL{$P@=sNe~{KK=CGH;S`~7AyQLr7O|Qg zpM(`L7H*}0dZuU!mKmzvM{GG%0iOg2%VJ3NT1)BV68EERs#`fTf=_~@s}Omp<^$F%VkO0dTlAaPB0;scEO9{d}tYI z-Py-EE%JtEsGk}fmtX0JlS*m6?HB9yXf>iFcJa<(>OV^Z% z^v1qc#BiDCuq({$SJ5xphE~&e+6I)oI`Pj3Yb$BsLv#s#?o24hlzf+QHNC)yoc`*D zeVYT#5IjpuFs3h0;lwMvE_!+InyCaOTil>hlmZX0EA(U2(efN!&VJ=!+0*U}qy(3E z2o}I1vV)EOQBQKssXRlCu(!rZf9C}{7rL!PiL`k0;$&B#DJ%%M>_0XTe6BCl%0@VF zK@6F&IYCq5s^y&8ZU)R4H;@;6EMZc^uo}rr02A2}yZ0q`+eVpkDG1>1O6}C<2oL96MMOXuP~Fg} zzN~ReeTLkdr||Gmok_vwM=DohwO`qi)V_SvJ@IoBN7Og7s-IVv9{^YUvELvik5z%* zo}p+jFspCiPzG67bYd9KA4Q_+??=Sliuy6~FEQ4tn3;wZn%v;yeidO~JOaM?5c0jE%DuwTe(rs~HpA*l{rDO3+6-{$?eCeC0aSS_lTgHu zGO}8!V>UR_N^$C*tOtv|%a%5?gjv`@b*0u7|Am_`i*WK7oB#R+*oDG+SP^oN-`Pv^ z%p5b>D)wwS>)Ap&*e<4ubRq#iYZ7v&AU>Nc4T#I%%gm3kS~&{7b43S^lSYUu%!Sp5 z(5G>U=@QQ-NyN%5=Of2D%*Ht+Xx8vsQY!~tguS6`V#jHPZ#yzq2dEzTs%-fi7YPC4B zFSGck^5<5*q+AxO3a`1{SJ@dyJrQijavQ@*P;-89Pi4Hk}|L7_w$;paZ({^BOI)@!TFk1woD_d z{nV3w=2GL%yIAYjTC&ZoGq$ZWKBX(gsS)pAD@esSR`W6f=dBQClbx}X!%%BtkJVC# zUouF-lf7nC8*}AX0^vrThK{ic6eH9^^KLxt$3M@D*X|He z--zVFk3~AJ+ahyA8VH_?P}bY;(2%%(3n_y4km$E%4PNGteE$sNv*s$(-!QnX8)L_w z6ovG+wU}a4?K&{2Ky2L!WoEY}7rYeTb!NHegD}S&HKq3`M6sa+Y**M5Ebl}2#8tA? zfEkU_o5g3CsyMOhPg~#qSaW}ktqOlC$B5tA5V+)asjBwKS}s6Y2Y-S89o$k9#}hZc zH&p7scUS!9rx^UdiU0o57V(et@jqig)l*e$XG~9H=VkRosC>TcXg*2lxd~cB0)0zC z2?=q114v6SX_8G2b>qJLG1|$=zZ?{KXAQ(NOP{cLGInjI{!zP;Hg%KqQJ4t%c^{1XZJ_&|p=T_QD-UaJlR(aug{yM(cP9V{v}_ zi)OupvBq#`8Etx6KuWE_0h6N22xMSy6wsqC$k~JPK#~iKdbg9WNXcGsyuTsW~ z)a*R#%WT%>Ee4I+^-vkZBXDu4?3kNq(}UpXCG5_vTG7fnsxsuJqZ4ccB#~2;!8naB zZd&}zEx*D+9c$TS8!Z)8deUj>GMUcR_p~yp&J_pBV|Wdg6USeL7u2hRTgc*jfptog zVOYC^FYV@`zF;HMc?Ox%u6Z^*Ol@D$lfXU44-5b+;NY*CSS9%S=gQOS2dT)BKr9*g zd!--~REL1FLM+pfN;wDz?v`hMd7FHK=hg+N!k}}t0beZ^?mcb$d&|pWU$pNV^s+sC z&AZ*6K9NU$dlii>YVD>f7fr^u1HS4BLgyo|X>%;bZHUOS3;j^)gU+Hy)LEO9fz*&6JT^dK(aq51eto#H#kJ(Zu z)}5j`YBJ(cu#lIm1&>z{Jc$-3=qVauNXnZM?lx)l zf1-HKk>3>O<~9|i^J=rDyT=dUVLTloxcHv=Iedpc3rxYZ^9`%MIArR2e#o>n;;`jz z8-R3b8nR8HFuPPE*mF0{p@G^Sp!0bZ-yqjqcxMC9sA(s&d-V}hBvI|cJ+{C-A?#=?gkGgrSgAF^l+SS| zkY1lg9PXGHxpx>}8D*ZNqU>-HtKxvv=M3e?(#>u`(*n#hO9vTn)s9l~OuPg(r3Ct- zkh>od@m7$oae-s&$o=I}&115aCG~t&d2z!hY3sqQ(-oCFQG_x?oTmEVo=Na{L!Y@W zNMDdWb6u&3_@UAdgSFRL50F&*uSf8Wiy&T;9pSC|ExrYdT!hkp{Z*NKV(aQlz&_ER z>e!OMjsUc(&UFyV!lK?uf*qHA>H$Q5KCi<#Q0z^LuGf8!iFiML)4!NpDeV>i7;uZy zxyyN#9fg4I&ztjXyy21e-E+O`^3foY%|tMzI7g~nj#L-UC^wK^ebqN4v})&+2RV%C z`szaI8>;iB$dndkN8qMn$=!BoQ*#kgIBgSJ5(Qn&Dc6ugbMNGBd)ap@cv8r2D~N3@ zrCAZAt^PTy1`HoCMxIyF*`@E6T^a)&BcGJ^2QGAV3MfYkJqu;BdRaYwI#6=ghqkFy z8+X|)Y1(%IfK{wio6jH}8=9mgZ(CYZ;|Aq*K!DwL$2ErZFqI ztiyOAd|q~_J^IsS*8}1eOry5<`*U>1PF}VLu3o0=Ia5TN@4YMI)sXlW9^e2}H*y!A z%7rA-j`na+!E7RI(%hak#H(zSq9&V=_9HINPwv%}cVjHgj1G04PS2wj&wTq-rL5=p zq=v|~H_CM@aCbvAd=T;L9|@=L4nu?@pFlv6-+SZ#^PNonH+Qmx@_WNS^Y5FP-%!uy zy`@r6@4ZFzulqSc)m#x-6~puMN3>pONKith`rJT#5zVrBIZf~$@_sXfuzM#+80tXj z*0!11=U2~H(_h|M9De7p^Ni;Y1-egU9CLUI&*^a(v{1n*J6c;~w>(GOm$+|lmp^$w zU=Kv`pA8h4Ic(b@M2LN42PjXVE`*mHmii>kZ#Mgw^MAD6=pHkBlRzNVQO;ZPH61!- ziB6W2p6$+8sE_ZC_c_5&wH@h}Lg3UZ8?iuOjb@}TLZMse=3nI#IgvYq`v95DFjTYt z7JtVzhFtuUdMUAlVeH3Qy* zlU1*s=z&lu6{=Em5GG_oQ&q1Hw#G}g)Tn!_iwP5vx;S)LxO!kXO37! zhq};^VebPHZO%QW7teRy-Ivdgs!96a1U)ehmj#urzGkj=Napj1jw9rj`fY!x&80>F)kjb_^XqRcLOji}P+;&6e z#iz3QYF-H5h>pWqm3dBL=jd;A4>%QmC+e(II}F-X9z|9Y#fg+Ct<1QFIB#kGD3iKQ z_9jvr$$nUpskc$cuw)7yi4O3n?0QpF8`6`)OnIiMLrP+DNF1t;vVPUH*V)p9qm{P# zrF2%6q)vB1cS(Q83aX;h5d3K&qV{$m`KM>ZbSdG6T)V`iNj`ehrXkx0QD^fHXt~Df z>Ei(#8m(65&4QY5g;py5j@x%d`gPa|H9?Yc9Xwrl66Qai9GP98s8Y)so&}d!xxO!e z*E^j7?)T&o|9rf{KdCOq_N967skbqx4$`etOAb-Tx4BFhzOG_E zY^@otGq4^c()5QJPgmwBRnsXB{uRt9+B}SLf)$wN=CAgBY|wgG2S>WxmoKG#XJ9a; z6%SAiJ67DfLI=t$^cQ(%IUog?sI4s10(=#w`owO>9J#6uGhECp-ynt>o{_(*SuhAzVVqfYAO!P92+g4g?3Yxn1bBbz9ckPUG9)MUa67^;Y;d8X!tL z2z|rK~Z$ScDSICK^11)iOv+|z@uiv zALPJr1u427&($M*-Kq^gye6sdcT23X=aRas2wPsL11(52=1-uUOf9Zl53(->|3u@0 zOJ{1OVFNHJRuAB~Gw^Qp&HmJqoBk$T6j~L7tsCLPE61W!IcX%i%UU1P=%nwO8(h0F z5WmrP!Kl@7yNx%Y@kqHi8QtIs%0A|RLyRTZ?5*b0Kn~9_w90>zXi(v|D#%u(jz#g2 zCDw!{acVAo-c>?-dLNc)I7XoMa&DDxf{_EGFH=|E$ys}KaYk8&B{HnlE5>Y_0s0Lq z+v^wD@8_$$vHP!f;2ONPnv_|4b@I?Du zsa2IIXKLwDF#x?ZTjxp706_Lt0+M@@$3B&;;KJwxv>ocqXBCK3g^<31c*Wb^XPbf; z(6wx&tQ@%-8=H8&J>7$K)8Ycfc08f&m*dC4P{HA#Lm4f^%q0gJLy-gzv)vuETsKTr zqEd%`nC%(eqJot%P%CK-!8aqj@1 z^DnQbbI)=P`9e0xP4Za$t%AzePm-@uE`7l~ls;)=GVb-rtX`1yj=IKVtTeHx_EL-i zhyCn$_^Y@mnxxWwS6L)Al?}HzqvLvyHV%BvSoPbM3g%5*a;>bv=Q9GK`m)TUZ?G+x z_OVRlC}yfSNoH^`M$73WUvPRkIyO-{Xu!><>ER(V+A+|Ey8;JlvI?^*DQoLyDyP*y zkJVR3FBLWmxUDV&yfN3%gE0$W`n!a}PwS0yZ&8Vd=c*-E&Feu2qeV9(juVP(} z*LZ$M5>i0>_xJLQ{#l`Plx>sg<3mV?BXt@V%ny%;_a-GXLl5>S&HXZh0aV`WeE-=*e^<=??~qGS&)!Hx&p^-c z|D9At?e~+4!d>T1O-(hQsG(V-U9Xaz&@kBlNm{p*7@!b`Chm@%_yJun3AWAig$$2* z=F5!G)~ozr+n7X5qS+TtPWz*E`zdaw=i8;FOAx1;GJ*&!V7`rFo}OYKjYUawhJd&r zs1YhE>Kdh!(z@O>XVk!O?EQ3|c934j#c1WqEzTNK!6lnOs^T*|W^9zj>Y4$0B5hz| z8tk=46Agx);~L_pw`>wazosV?_3}2fOV*ZpKSz{sx2iM5J{6Dgl^A#Y&DNWS$20JX zOV1es7b{04 zvqjM02l(lBYgi9oM-sv?m(6qv=vLT)s;s<$9w1ksXnmGv0L%pw zJ%lj!gD|oQ89=$>r_&%)>nH0hnmM@f~B%=CExz0mO!h}ZzMvAlrSY{tVu2f zB`)QUK~VaD^_!XFA6PR4-Vd+G9}n;Emn#46@G3e29sWtjjDNniWX(8|CZn@zm|CBa zKjt9K7{}}Y6CEg))`(y#K+w=o5nL6wWcZa4*(Aq1R0+{W*v7IafUqCDL0AlCp_V+G zd>b^FeAgHH;<@ri*Ut`DSEk$8P~EO}=cCPcsvhzFV*lCB^OWTl1*S$Y`8gZxC$L$! z59~wy9zSm<@s{rAY()6S>*|;5V3a2b5Z!@1Kqu*z?$VVS$lP^vAlb`9Al?@2w&p`4 zo~)V6XjBnc@sKk!*WQpb3s>2W8=#Ypg{Nreljj-SSDs9Gz9Bz=Cq_5nizz_%p@!>_ z%CrcBo;UdbuU*}n>q|=JZQc**2-(31FBGHXgrf(2WKG<}5gx?& zlvi>I#VD!qw_`)hU62pLsAL6;pK~NrvPOpTJd;0+qb|G<6v`qq3Be@pL_S{|SNoE1 z_`TleL2h>VaCRM+0`Yj&-If+}s_{}>@z;^oxg{$rgg$U?IQ6Efd)0VpWVF1Y zps?PB3i-Xb4B`3`K5?DDl7+QmS=a<$&wT-mNQO7jJUSZ6>4f}?x&tf8`p|3VzKV?H2Ho>l3QJm+g5?rm59sl zRw3dxG~T!Ymq3QH9~LVQ8yJ@wq}iYPoCb~Wvx+os>YxB#OPQFfe3BmS(+vtQ_>J^q zv!x1wTvF6wPbn2o9@VV2VQG+vWN{mDvE7?dKZ}_6e)Z7#_^3i@efoTtHyD8}=fqrP z9C?k_>^R-H0YznWp3x35j3>U zxwZyZIgjb*M`^-ZphyJGN7Ps6QHY3zW_o2auiv#F6(d8J!{1hjB9SZ(9irn?%l64f z%4)unw45Z?$9<_ewCAwdS5JIKNtMDMY2WCVB13l8m?TK>H&BWV)l@eLf;79JpEW;p za#U~IxWG`UGe}AFA9po%Yb|UoWlCDzz0GSIBhnOHmo>TOdeb5+;z+!kT;hUsHzDhj zG~8o@Ev{^CenEy9GwI;ZpS>5CQ(01iT069jurHaxs{&3#lf*GyC-2lKlg1Cpm?&4= zGBiq$x6)YV0+rJ!qlZu$GWyNY*C1ak}`VjF3>XQq7rB(>qg&ep%;hV!+5s_@VZq9W|~( zdPe7=jsk|w$BeU8<(0BEDKu2mcIHOPl!5SS^p|#sNlF@ZgWvUd;`)UU*I}c}CQ4Al z8L`(MW|PZT`~=CNacNsuKUwGC4z2IgF(34FZDQYomLzLtPVKMzvr`sr!F8s*sml6V zAwyH%>+9$JEL;|by(3C4O6&+uKk#?l8JRU(dv9V*q1w7`r|FFh*o2#L^tG?{PNYJf{B)uz&?@PJDO$_A^$=%1Da_8mY zGbL@%K_3%tqm_wChYk3oSZEaDs=kQ9S-*jF@iY7+gqwvUCK2KJcro_#YVFgy}(ZPf*DB?pj z!QFmI(ffN$#{m~d5@m7wsswt5x`*8+UQZ2)B2xiM?s+S4;>>*taiW<{N4AVrsxCqo zEf#5vVL**!El|*m_E-g*AD-@3_rt&nOWSs6`ZV96e0BG$vX147co#-0m+$t5+k?yA zb;0d==loZ9*>$P8J%}h@%c>i@QFK>xRw@^1t~%Pdn5@}5fyVPqOI{Lm7d}hWOCKc; zss_pKFUeWjU-7@uK|f$ z8bn}5#EK(qyxbMEx#_Hq@?IGx8!{oO8_YFvRL7876H3(%iI&ip1)hanX8lg1QHDSA zsweE4*Xdp7hJD%0=I{2ha6vQz$+(IYJH!7u*Jv3e{*Tx(SxNA>}8CE0UO!KqK z4|-8=foxG9r@J)(bb}EJM-&FG%|BdIQ$E1E5ZIyQTEm7tN9DK=AwKw`?iGiJ3+_=J z9~ofJMH1Q+J7Oyj!GHBMe?-rT%4_X9{~)y$HQ4yfz|9efX;2LOVG9&>b>O3o+F6y+ z)iI&@Aym+dk4La2KOWEfowQyyc#;wYXXvaXUyw{u>K5?1M^u(-yXbO4$OPSN{uwY| z_ES6A&kiGg#b1L*RqWcHKhJses0H&m;(j!@c&Ja7*E--VlF}gOK^?)kgfxH)=<$^pbqy=`a zT`-L(+$0bC8viSYvD`SQ0moK1cseHj*R^=Xn_)m_fuaJBAr{rdOa>-UCen*%a(|*F z{;(QuKb+YlLIbQcQtvHt+jkhzAD)82)uuqBRSw6LqN4X8bWY&Dq%jn(F%+MbE*!fr z7l*}%NX?x!5!;DbotBuKy?;2enHl05qJ3u3r62(r+ACz{3@IY!7?C=`jKP{08>Lr!bJWvIx)-5TyLj1@d)gSozWV`zawXZKc5zjudWL_Az^$e1G`klw&*+ zPEqe|;mdlqWmxJB6p)>PXei`G&T}@W!k6yPb6Uy_#4j0E=TDbH)@X9~5QSxaGM$x% z+%`T4c9cpmP)V>%#2&GeJJgTUa!cwfbT0F&YML{6Mlo!RAh(ZsL`xh0B)+RjMo_05 zaX*5)c7MTt5likAkBxM0&b_IY5QsQQO9qJQnR(CGalib2nE$}ig54>VwHX@639LliK=)a8DwDpX~otr!+RD_ za;UiVaevhf=W26^*VA{|^SrJ{lSNV6YCLM2!Qm|U=7>$k2p@hW{Ad@**d>$IY#W;E zmMpp{etTi((>lsvlVCTZO`t*nc5Cb*H*hDYWBC1!c+U382`{SM3yD$P#U<`U#8~xh z8w~pzW}0;F+kz^{hm_kp*YkU*w~zCqedArCTC@=9OknLbJW|}Cc(#<5ndu9N;8Y)! z0|+le3vL(y_sx<<4SBVxB-scJX38ixgI?i8o@XnAW0X`tM4HQ52^Ce{Q*JiF={1B; z8@^vz?pPZ){Qf__zBx$pF59-N%eK{Jb=kJ<>auOym8CA*wr$(CZJYh-`)1yqckjD1 z5t$Jgk^f|#v(G;Jx7J>3tJ(@KEr+6GU6GGQlD(W!jQ7p87iGzl3bXM>@m%HE95A^8 zt+ zM%=jM-ZyKiQT}K-|6bIr|CIrSaqp?T*T=SI=F0SLZ9JN&r0MtG6dP#y+M%Wq@n~DM z+8OWh2(Q+Dv17f;s*^;uv*+ZZQ(=W zC(sr7?7mCT%vIoxjOV7Y&&((6K;roIvJK)J&E&>~ZOPE0;@FzkUZ2v?5`oIDG%}R3 zJVIS|=*qP4a6$88&Iw!OGT%+W>$`B_|ZeN83Rlr3I%Syo$dcH>>eNb|+1U-I(K9IjY0aicZPo(e~ z;hzI9xsaT~j9|jy_PPy5`N%}bqHasj%&|{SDOe*Gb?G76`On)9PUD5GRd)(`lM`O> ze=Y~;u{S1IOb(Z6mS)H9mh~@!53vKP+SyJkXgs>(P|Q&a||rvLToEZ&c*`x50NCfks#`TWgC(R?1su)ux$ zmi5)M|L?((zf6h#=OgI9FwDO{8GKpo{1e4g(NINJMfoI!6vr4sAoRx)I>;qhZ&%jn zffk8h_BAE1P^S`PfDCBa9v6B6`V8W^FVHey+RgDQwt!r9uHbo{9;*Dzob8xOCxATc z8PL!2;`H!&vj_WPqoTULrS|~B@|ss7wb%6-#)vDBS}JxW4U^Lh7^e1RAfPFg)?xN& zvYwVX%k&X-=$rc2m-IaQ8s9v3fZ~EFI|qmwz0e2u=n{V|{?djdKo`*FcO{Jzr!}1W z1zIzn`YDw~ev+^&v{*APVW-gZsnnp)=|1plR+3qe|1BbAeD{7Y zL~WK#)e17?5_7i<%?op3TlOMzyMvZP``$ro68;S;;v;Q2TZWnM`egvXD8TAWSdbVY zNefxHp^TYI6p(GS+<*icfi^CYNtaNDTBY|1f_}+L`9fN^G!Ozlt7uB2B+J#!ZV@?y zQ%^O6WL{T?o-5b{vow(DM%l`jj7Kh4_x)2k&cmx%Sayg#2%x7qOXtfEE67&3LmYP| z7Nl!Ro?W~H?VPM{4|Ifje=h9YVVrmi5Qct~>!Xx{EuzEdO$CsVn~B*TGf*1(I`3X4yU4IyF+{%K=%5;bEn>foYp|edWO5>#JxP4& zlUZoPD@i+YL@cyaxLbRs41zvXak5A~g(sfvqi8&RUxn4ib&mi?rl$1vMZSOFkjNUL zi_C@}%HC!Z=fNR-5emQ|YU$!%JWg_!ysp>Y<)WBURLDXwNxp<35_b&DM>+v^bSBy2 z7awfemW;IS$--F^*p6ssmZ-bAHK&B7QY_wdbY&SbGK|`6RxIthNOIx zUt(tHotHsZgAr!%aPk2UL^9OJB)*~)R`?y|2C%-0ZN7@nQcSlvs7K#3bIECR@p*+I zpIyxTvnF@51b|pT)G^I)GD!zW?D$8vRXamd?&gx42131PqU`V4g#X<0 zkVlJ_d2K@2iEKRvc5D}YLJ^GY>Mh3MGsNb$OgNW?uqI4xmsQYBxwN@_zcZG78Y+=R zW(t^#e}w^J3m4xtDH$`+PY|+)Y!p><(`r7$>(_xGOmbAsh2AeB?&kZnrzLU=HC*R@ zTOp(2e{_dvK<~^zADuj$>=MT$%G?sDhik~%1cFGH$T-RE4*=QA^|cK}+oeRat;lI| zA<;x_vi;S4k$VSax32LzDzOcgFqy`C0{dQZTzzE{Xb{XYgT^^7NXq9u73&j4WNpE@ z(X%v!hmxc49dHOpp!0l}PNLA6WpNOlcqMj&XMe0!x=}jweV>T5W4#k?D%cj%G`hp-20UK-XPOY2}fIgYsoALa7&jirVs zP&Juxj1${!d`+!8H;4op6?zONND)s(A8`QYK!hxYUz2nGT`isBqivD6eUhlAFFI7>{`b#zvbQ9O zdRiXv!ryB|Hjd8}WQMY@Ibq-3?v?wkoOJaD_)*K?EMYjwb%*@|VnL^Rup}H^0`BQh zq*v9459BXA%HmE_KeKM~%9__+)S{*>Eu0pG$3DvB@;G8(>S%cRK(dUQvL(yJfz$Jn zXUA6tAO`g|9lw)=F_<+DGj#g9M2u70b9ZK-wf~?<`m;~Yb(EZ+LSz^w)Gp0#BvrHw zV@#iTi^&`pN%j{k+%%0!(EmkNumk_6nE1M||Gzu=FHYxgSops}LSBOwm>=b{u)^9Z zNo9jpIL1^m9-THY7$Bsez3Lj5+?WTruw-b{_)H8$fdA^8B$+G+fE&eSuzxAe(j5*z z_8wrn;Z%ZSy2~R7##c}#G}Nl2O~js>+ZItfr#SryRNHpAwe4C?l}`N)vU5V zA%{mdHrwr7GBv`l%Xmmdqh&eg0XIWbXSvA4FaP@!1z?uVA{^`(IhRRe!{e90nu+KU z!e^!)2lAa}dzu3kaSnqYEhLvdmI!;FZWDd2V1rKKK(rdY-zWYi6ouY_HvxlyGKpl z)?&kKfaG^>?m}ym`9*F+#bTp+o2i9{)!FRlQD@rF{voV*+Ev=0jf3?+?@ya|lU}T} z-Y>_a^v7*_lm!)fN|z?fgA3rz_dogsVc@an(ep1f$Aitdxb>&= z^j7(!Fz=iBWdlpn8^O%Tc(Y(;72n_Suz7I{O^)C6+!HgQXYar|x*yPC-M_?o$3_5r zjQc)1>0Jrvj^1m^J_{ILeb>|-CjdUYK^i?My+8XBO(@E*2f1#rRe zp17vzv%&BlyXFKuVQiW_a|>qf00VgG-eY!TVSJ3Aqj!h^U3BjWI|iBOTmJVh+9+Iq z^rp$cyv(P0L0>pE$_pB^ZwqNGQk3JWW5dnW>rPzlXQcT340cCcofETXhv(-&UQQKo z={E8xGV3+$CD$$-(-tzL0~G10>{1iHq+HCewc&f1{03>Q_CHEar23|AmE6Up(=tXD ze_>+7i3o$WC1l1b%n$#C_&%ivFMK_=q7G<6Ddy`XLe@pEhyExb~eIyhJ!G|o&TU55H7DEVZWOpk=zFblb^t?mY; zysz$U409;30|F^KLy2!b)_h_Bjdg8hpajkZM63vPIsalZ5O-{mi8M)%XgPaAg0{9U z=rIBGF(NduV^{(QKYS%cDn}lI0Y}w-4~*URRTra?0h2SsO7iH3OZ^ZHO*C`^%KIW= za-e>Rq3C)_vzjk#q*wq$aAA*xA)7u&h3ogVGp0!%&R}DHMy_RObs6(3e}*WuL-^Gr z$2?0H7#Le4zvOEDU#^HuBn+CP1p6pfJ2i>`=0xftSGC2ZC5NQqA&oUXvRbrzQKh^j zFG0b8dK+C%<&~vA>q#6l7Vvt+qI07LKI>%OU9OIw<`yKf)EewYv}LUvX39et z!5G3E(a|w{6xdo&CPl8m+D#r4y)xePaINMm7BA$-nbFt_-~2Y4T{@A9o6L0v_@v!a z3>MSl5UvJn%uK*o&|wAlt}7Q8DrfC ziR;YzM2`gz=Rr_;5tuvN2nA;h`m;BpCXfn9=~7WqsR@up=EKEZ=C?_y%iO`r>xU_D z^daylJ_~>Ir=(&CS8NdiFT0p*Als$q_c+xsff57(RsMh%Uq>N8WD!n*fG)+LOi7!U zHeuYCQXecZ(o~-u^M=b9vnA+*0)%6zFlhGLQuJW~#xQ71n*Dbq0A(1J#%fcxuzd|M z*qD_D&6!U}U~$&=-#8mmw5*MjG@9#!Bw6P?*o~4Cls2+EG zkwvdKv^)JEtg7gN{mXT)o}PG44w^!_xT{|oa{1`C6YP3nbOh9qC)r_6c=@94TuZ>g zEeNi%4On!YT-L;GrSD$lu;%miT}Z365EXE$e$p=>Uj1G(`{cQ+R{HYKay>Le?dcTT zQ^HyIW9xMYt;h(V#HC*M0@z1waC7d33K`TgTMQ)D7gQuqoODdrUZ0iVxHqu&utMV3 zQxhK(ze})z?rhq!jSh=-v&={Gw{xWbjLIcAWRV9XUGtw$`<7q zb1a&2CFM(Iz^itRnq)!sGFHMAC+FIW$U?rF@nI`kS{8V$DQAIQ(=()Gg zK=(w}NviVb9l0?U_fA`YxVX13LEl7De|MVSFHARVp8>hhF0;4O!6}a*%WaVggVjB-vA231^o`elHBGb)t>Pc zAz|MvP;((h6H#^+&#Rtnv9i_;Jb@_nk!GtXO2iHy?;sDztKKGa@Q1|X*xa%-)5w_h zI!&AD$sSW@)MHlghiT;MGt<5uvzk(I|0p_j09^w_#Hlo z9WL^`HeC-|%ge%0M|fPxC%r}udVTD{efn3R(dhs-TfFamef9pY2*zt>%-n>kjvT$F zgd#38NYy6Q7FzLqW}u5T;fbSint6*gFg=+d7CRh(XNU!0E)E)5@MEA)VlHmxAKh~d zBz6>nzijDLW$bV(wY~o=bKw>%a>F0dqO|vUi}U?ZJET7Rk+Gxv+sz7sevE5CYO!R~ zH<^`t;HS%6o+QG=97@Kv6+%0sZ}&_u$VW=6e>$&DCgGZ4<0|tfH^}%~^&nYzt+f)6 zl~7sf9h0)lGXd#KjHB94a!VX*^z393Ax!JH0wi&dootedQAH(_50GA7-0J-CgP$j$ zu?a0It`g<5Bc0vZwBZuzXA6R8W57*>3Z@mCm@4`MT))-Da1ibC;`t?}3s( zc&NetIoFHm^hp56NiGs}1Q#3DANBn(P*-ScRKM67m<5{CKAe3$XfYNdlNp`M;kf9@ zuAsm|v)HIsE1TE81O(OL`9Y$(aY&hPf7i3iNBhS&#M<=a_CpaHPLKk#r@?} z^kt@1^F%H`Zx@s4THx+l~IaUP3xblkoVJrJ#C3cy$k-Bp08U^*G;+lIBn)$`s7ZrD5GRmQm zl28(s_ax5ckyzcuxztU#v61Fd`|(uuqheh%iuzMHR$8K6wdF%QD*LrbZro`@hmOms z2<#F_YnBq0sYrIs&+P7T%M`Ulcxs%xXPU$_F{NztR3)+s7|Vl`VTIvtrBCNuMij?Y zol2z<9P)~i)7l4ciWt|y=$qgPj2&VWUtKfK@hmqx`gjVy+)HK~IL?5hpSKZ;?h^@- z&KzrTqo0?nNvv-Y)vD@vU_+;12yTq0WXo1K795smkGOd&tSK0Vz%q78#%~VvrfwDq z>&cGmt9&09^yW52ny6M+fK6rRts8eA^b}s7d4CgE@hS-uzxA_3oLBfKAfLib|2g<$ zwT&yN*xsfA)$@ly37In`o%7Am$)gvvJcd@r;Tb7tOby>FqD*>{TO#WT9Mqe%jrFxn z>k*4p!WMrH+=uIjz$^IH9zc!18SZ%qP#2d)pvg;XZ+G>lXt2Y6-0KhMUk_8YX0h_* z&|eM@U*5m}zUKblnl%5_{q@g=`+pSQ394s4$VO=1h0+{m$%&1+7B}B3#nw=;g!zKh z&(WgwDQKueT#82OGg2q}Qy|f=JmiQ(SGQOOq$?^i*V)ID%_F5vP(IxduC)*s1Ch8G{lEuX7$v4U% z(cJdF$Cq=(TpM8zFD6;>1L7)lo6-cg?u!z@~QDF*BEL{G)Z^cZN8;3GZa% zo-ws0c8%I&`=^XtQLBz!rH9TqHm7f(al5`;QDLp>;ucHH#^49-3$HAW)79u?Q%}-Q z#tTr*q5z8&9+UI|{w8K>I&M7aSOnob9`)O7iA6asy$^@8(4a$bvzfUCeMI&RSiq(w+X%P+#pjA~qq z+G;ifx4A>9$oNV~R)fRHv`p%^5De%;D|a^suMZ_FwlRUZ4xDSLPfAnS1S2FAdh6!Z zLlJKD8&^`8rOi~RPIv{6L25QX`@wr^j=6PSy2%*Ud_98)Cx|2YG1~p_B%8Y>4S+M% zz4_?mx_BhY<5PTy2pSM)8g9Ke`^-{8xFwKvB6cGoUHRuFtpu&aw2g&%I-)g-j;UM5 z%nW$TByDR#6J_&6J2_+NM$PR(_j9YvhU3io!6Tfdx>_ueSJkBAGTzLTCI^tPU+rwL z(vOF;0g6D_M;WPpR4ps!ZzLV83v+J>Hl(&i-=~}p_X!e$f997CnH_z1>-jP|Lz5nw zM7J;8E`TQ&Nx;y+GFXUS_u(+0aPY*nnHAo8@CS30j-j4i*e!GNDu{vEJ8}PoD7G+X zFZ4{v+=^|@bk9s)u(#xBJ4)?}84h>X z*lxA`^c>*#)v@n~0_N0ivK@uNa!vNW(!KE)Dw_hnsVdh&{#67BAR!so|E`BgUPZ$OS)AE#=8b z_3*-rcm=h-%7uZ!j5y=TNlM1&)=`iqCYAaW*;4O`)Y9|P))mtsLo*XB9gPh@MG+L` zBsV5S9Q<-my9Pudl(Q~wW3w^Iib~$l;_(qdv=RZ9Tf>cxYFTHZy3Z7pXg@^$prnUB z5b0(<_Htcg`o7AABy9aP78pAeH?lUebaQD0DcnLa5>>5aNL(T9YJP-SbA$Cw)hXcS zBls%IJK9Sq;UoRv&A|Fo3L}V~z%7wT2mK&7FViG5bbo2#N8Rk2d=j}uWWq*XS?Xvn ztIccN;t8J-f`-VreA>YUp=a;yfXUk%`W?`Obn?o%+H&)$GoeUD#AXBsOgUy}2yTPD zb*O%#+MJ}qd{?7D2%7jU9VU~db$&G6r!h}f9`Mq=Dkpj3qF+kb>I^lk6XYl2a=#7? zZ}-IVZ;k2SG$n7k8+%;~z6VKJf=!iJ6R;+#+^Xht25Lz{B~)@Q?y9 zT0$Cgjj(B2!kYXRY`aJ=$Fi(sVACS#4o89VWt80b@@0fy1Z8c*oiB3X=U6P zhvbizM?bkj=Rt|XOh4pqLLSkC@fB7D(uMM+4cc%*_lL|f`tua`U&3Jv!^F^P4m}~s zVYZzeN3-+jMib&_2GUN%dC-1tNRV9ek~h5W4<%u)kWcLW1bfvo{1x$7dsK2M1 z+S@-pTp`#rr(F`Y?Ha8MC>Z0wc$pcwzB-V|?S(HG&hwIw=Kx#M6GU9{g5-AoT7=Q^ zRLXF9QOGu;-bGg)Eh65Nqlbhimt13SyuO>`TT!BLb@0CMwj2gp{G8=HDz0+_S6^MS z=14p5=&s;vTSGOH>xhJIS%hUxpl8HVlrFr!J*wrF8l*|m^NagB8Oxg7sfjZo)NQq{F|}!H4o>iux1{?ZKcEGDZTFKa`WTiZ4Le# zm1`wo&Mn>e^gfR!+B%+uIUBJ7te2&(fp7la&ukU0S8i|G&U&Ay$LwS?SKP?P zzjsC|Zb|p^sBb~QEr=ViT&Q&fQxax&!+NntiQZ-(MW9$?A!0FwP^kGflI)?&0W+5q zJe<2Kuqu!3^-!>`iyHUR`^P*pS|f*qRMTS!wQ#`@;{$wt0!hCcRU|^H@MSAc+hK@4 zHC`e$@+y)bmA0KIoYPrj{<)E2OyX%pCvg-@)bI!K59^Cf7{mcYunO-~X+gU+z3S>4`!XV2;(dP%bi z&wZ%b&c3&Tqfem|sXv}pC&{J@r~A#t4b_`?-|d(qZT({7v7S2By$NQ{zNPy2B52~O zNk}(mr99^?et9&!&D7Xnbl#u%+$fBkDd8$Xc_|)*38-RvI*~oLibqJq0(NCCa4MQ~ z0tHPj!TUPVjBI*AAq0AneBv2ED2sSX3JvU%9q@JTPu#wC)39sAu)5^Os2Q_F58wgX z5YN-lp^p7{vSKHo&L45yhysydBXv23r<7UmqYmHqJBb>7HkqgSx$+#Nqup_dDNgxd zp=j?RkKB7(GOtUoDCN&}uc_*P_*6t?9pBgl-;D@I>p&e$>C^oFgDi*-{5~P~i3{)a z^tT0qW`9~}_!rt*^hL4!`|6M8e?eUTzv}OAx#j=G)gzSt5$2MTzyz~UP`5e$Xr|CC zD2V-pVblIH1DJf9NycUY1bVSw-MEu{88T*Ws;yza4EO2Cf|H;QQQP(9WSdc4f>dif0W-aN11w%2 z-3Ow(^LAY)ZM}C*0bRcUAv!D_&p!os0b4~?^;t=qWnjFIL2ZNKMg;U@q#wWZmQAr8 zRS`zG0oSPp5CGGj`T>;#Q=R5E&n-epogNB%i>S83^`!_sOeSEA`lv=}EVL&s{zRp; zn|?;e-*{Q&Zr19kNoKo3Ucilhjw;U}yE-?eka5TFMSB2!2?YF{r z)v&lhp;6p_$fn$n_UdBTPB^y6q!aF_U8`GUowQkiB&lVdb4ZRx_H5Z&94t)} zMt~==U!D=qwMPg`<%z}0_?cj2LH69`aC&({=A`W z8o?Lf;*-^7sFo(+FN&AChXciU|69#Q)god)^>tMCzb=A*9hKAnF?tO3zmCd3=O0D}fIo4H6DH%lTKTi&P1A*0D|5k|A{qu+J3vl31pU-za4A4B?;{afAxOZ@c zKbW5ZC4Y3$5Xe6b^$c~l^{0U-z8^q>M+U*<;xOQ0qv2Q{A|%?g)9L9Uhx^vXg#dQ) zfUVkh#Tl^B(VH77*<4N>Ej8_3a{(?h_)(CuT1BVWRa7mQ&HWAl5$nyAl%5jNw~HXM zWi5x27c0&`Y}wW@%ban}-P-#fmB#IBvBkoyh@rjU3%*#4wggzHzxc)U@7EODI|A z0ezz4E^>0W7v$-4BX~%rb>^J)AbzixmaUgR_t&(FlQzjJVn-Z;{3MfcS+mUfo2c&a#}#96Xm>z^ba8YZS}0g0K*Leb)OnFk(cui8 zH`wghq#KOsaoWaqdKV!NY=~NWH;e~pN!r=|@1;#k(ffq{V#@};-v2%z|K-j2&!u&+ z(zCSu?@@_jfMEDM`R)6^f>KZK1$Yo9PZtHo%XYil?+@Q8Z=Vm{1OE8;@3_&#*QU+snfw{YQ1Z1UaEWzqG9*MRQ^qp}Wl0!-oQcf>Vhb5_r}j)afUcLd?bkcF^~({Ts-k{dgaCJ%$sXlHknO6(9)kKrb&?JJNiM+Mog7jetb*LSeylm znK0Hj?;6Xm%=K%J0&D`vqOv=!=oBZh1>(pIOLoZ(z~A5~ek2iLf>j@FVy0;%7ifab z6?}xHIiY&YCA)`}GPTsKvPb3AIhsAt-n|{gX)+);+@}+9jMT5&8uRbEEi9V#X=~N3-G% z3)p1Iyj|w#0nF(ZEP@qlo&*o-hLm9SXP!zxQ+45uO2nFB=i;{W<+N*BPljh)v{jO5 z(jX^H?Xgva4GZ}oC-bIJ^aTEN^IA5~CUq3IfLpjFEwZ{=hw71rQ>XNi7l#JN)e?K5 zL)2uW>3W$%Rytd@o#tME6si_$)8(?Q#v1@Pt*PQlZLZUgV%>R~L!X(f+)&3M>8n%V zEs;@LJxRhChw9|K9_d%w$oYlTT8Hzh%-}hUVu1(gSwHeb#>u>IhV!m7*imnX*TrIu zGiO}Q>p2VO$`Ow4t_t1x#Ttezn{(?}jec--W19GtmE zrxHA|j9E0Qm}{6vuzD)vxV`0` z-LmX0RolnqVjBmV>?8)bHA9F=kA>Ii7kD8>sXKBW8jG029L=n2r$Yn=eKpD4+x>PP z*Dq|sDQLo^+7JDxGp+a05#{iBcC-ybLODv%>nz0sLCuvfmO$m~N;tR?`A!<*b{AS#iQyNIpAF`%8*Ks4<=4MSi|#7+#7EX z>A{DR_0z;weS{()vOuJm7SL|CXqiwei&?J~Z_X1#Vy#D%j@;04Hn@4;KendoF^u22 z_3O`m0RKuE>fkeM3KSleX-!mK9!jrAb?}7KQsa^fg9weloyAoP4o5x9R>m~WO&sAcTZkH7cLW1SOWgJo89&YNuvzV9$3E3)(QL?ta7n}0pta5>~l31 zT$ji<3d#J^=Vjs#G0qYju4Waq8!-!9N*!^ewmYvhYyP!li6@P{Tk7e#G=$#~M`hNa zhRw>z{1MgN3R?MWjVRC|_E|gK(Z4m^BpF2SKyg%B=G%1JC}T;0ML$?v^ye_w#`DSW zHmt`sMlQY2Cr7U;CmzEb+;89fgfunvjN$SlOKEi5K!1r;eHBR;sKUA@C}2Oqg%sxR zQELL1s4%ojEIEwZZs@H%J8Y16NWDVv=uW&_r}hQY=}wwXBuO`fyFei*RGbiTLOqQM z11#g-Fe|BCoRr|0PTKCjowsWuvYZ&bVnZCpi^I|wa*`0o+(3%2=bCKs#UFb?(oia( z>(){NsF{%nV#CFXziUbH44Q(MhM|%rbn0JTU~TE$W}KV8^T%E3KR2seiJY3hqUJ8$ zn=r~uc(;LhJMPj4J|(AHy}^HfmF`TtifN-D3@Q0!CD)l=9B9KLQ$_|~AP8}4Fr$Cb z?v&ko8)ENp^_zPZB<1bKXL%udmj}@_U^oB`8G*kYl6pmcRs2DjI3zsW;fOz*LvK9G zdJG!DYkf=h9M|0T=t>$;$E+_aJ>EgHTzn%)>Lal+?cTODN}8Subj=SQktUV)Zq^#2 zVaf2q_+C8{b!{Tunsym8ljl2LNP5mCLYs0=+3_tNth+D`1XKDm{b%y07-~_tLmJpE zKdrdEJySIo1OEKTW0CYL3ZAFGtoi%P3?Eihx$mP+Kq1!G&p>{sn`?jiCJ-T;@=pBG zoqsh4P5R;A$(ULWs(YZ=7p#C3Rz#L=OD`tn7e^IN6rcX?KOze$(@uj|MJ~lW;(pNX zX~;4nT?u@h9Y@jFurX-}I#(8E<=nEuQfE7Lp`3O*%OFa?iEV=g%v!^5^-DCvA2up2 zfG1a16x1Uwd2YWzHtvf;4i+;`2JU{3vOFl#!TWoY4tXXKh z%opikgf(9QWtDKkC{C+Du=mIt?pR-z!FJFSchW%8c)IKMgl@U7z zL((p*%0x`Q1C~ATRydH!Oy>(vct3YlMt^tx>y>O^;KD-{oxROOVCvwAC5JN;FGZ}& zp`H$v{W~!Nr~>!RCR?-wp)+AY6P1=&;aI>4U%s-r?Jz8IA$HjMeCCsz^6aX#?`g{3 zZtov0wu<3f)9;cYQ2_|^X4h8AT^eF#OdjzUAQHB45AG($LW}4TqtRY$79YjoD(Cbw}_6uXB{8HsZdmfdqhF`=`G}TW3>9Z zle#VsZjp$7Ylgp)u*-A?hEH^egeef_EQo%&B!RL`yyZi{&c`V?e7z~|%{!g((jLrM zROrlI|4g@olD*2!OGx>Sw30MXYb;M#%!$Hmz&(BY(cI7pGF}pMHp*(PsaK?g0}8pJ z0yuMSn0SHYF@4Mo*$F|lywS3O3-cx+C59KkKd2llM>bA2>xY-fN}0Ms6hCVMzk0hf)cwGccUqRI&BQ6B?x_@u4adO z|M4?jHMbuJfFXY}ikLdjEfqI47hln*ga|7;AsKhpMTVZyH_GuIP#&8pfV z>DilT@Fa3xmHVq6tF5dUO&hnN7N;y~)*#sn&IhR6Q7{JCakkE-q9}x5_G}?)#A#5CnJ}P+gT&KRx<<-cAj5?YGPvIv;mdW-hJFHulfaCS*?=cYM#R!q{(P;3%GBx43zm&++nz zHy-zImF=-IwlQO^AXQ2{Kv(;wr0{lau4XE4Kb#zy4o#gUvnj6$gHAQR&mg7-bf;lg zPR^hz19A1Ze<-a1{ZekOoE`@MPc!0GQUecm4LHP=2~5{}yVBf7v5|a8)Ee9}w5U=7FL4 z+I{rx8-TM5O$X>lCg}~x?hky~q~mglT#9?-X`X=NIv~-hS%>Xg zjGr;4&~lTKCP}C5{+oKQd7iVN%JDK-g+_OH=`u?N_KtmWJo$o@n)o*)8~N?e;`gXn zJ>I^Z5>p=M-hscqU9mMKRhCzFU87u-RcT{ckIdX6hhP~#gz>KuOR|=d2A$^r*I|GRUsN2o0>_zUd5?s>xu^#2X<=vy~({nS7;6`J$Pp~p2`I);$|R$cSJl^z zH2q1>MT6JxQfhK50pBPv!!8W9Qg{6onomKunu7Nx@k`GP8w+B^NF`Go754-~k{SAPhBNO^bU!j4X`1riBFr17y9mXJ+n#Shsf9Ice>MCz?J zIt6mH7&^oDRqx#_S?-=Euf^g%YM z1H#k=~zlJ3=mxs|@FQKaGnSmffRp<`2Q_Dbbd z^pz~OaCgCT_2)ycuuDg05gowi02*%N%zpp-(6{wc~dw}qmm-8z;o;S*A85w`K1llOgOQ;=sZ^!$!pM5ZB(4X z0J1Hw4t^|O!GNnz41_JDFu0r^aTtCPU>YZ?_np&v7;nN%eu7HkjIv^X6Ghai9Q8~| zSWUsD8&CCthWZrr85M25i}vV7J7RpCi|sqX`K9RIHGB#kOZ8Uk{2Z?t7Xg z)r`3h4)SDC>AAYw;~ymC;ke~Eg^qak9UyUsa@lX^V)*A1r z_9OcHSJ|tND8KbM04FQ$8G5K;cLZ&TFxtVsT2UFS z2`V-pynjJ=V*^09v|o%DC(-|lFa6J1Hvb>Kr1CWf`)j_-Kbk?Tj`E4S%JkEuhHi#^pn;W=j0deNkqaK4kNKrR*qUl17KHMjc5m^ zyzz>{gpYXwZ8T@|bmEWszK|>MEKPqr=6EBvE4N#v0J7!Mb=Cb5G82X6{c;3g zMhq7{mR!sN^Yzw!`Jz84-r~f)sC<$Ni}}Okg^xNtna8o+Fd3x6aseVjGORoVdT0(P zj602reiBLqE1tVC-@{iQAohjQMZG1s2uBc@vtm309{uOH>{>$pFir{q4T#mD(2s5X zwSiL(k6LbD;GolgY^VR)-G8>z8@Ya&UH!Fp4zvnJwnksG;SK+S(f%o(QPh$}{&IAh zOKdEc^!rX-0g8rV6)uPH{xdryHJXX%8$nM~0q*FobJI$OvbV2KZ$Fui>kU70K`8O- zCH##^u}kBa=<_J4%T>o;&Q6!(@1JjbWWJbO%F_X!FhOi#Py)%vLadj=`}zJ9@%gN{ zeZT!>))RWl?+nFA3wJ9v=gc2{XOpYrt(0mS<8=LDYZBpXJRnAMTt&bxK_I_X@astNakiOk= z_fRiYQ#Z|Dj$825{>nd)&1WE%Rq!~@IF9AAUkQrJN?)muC>vt43CV_5yq93oOXq2=b(nQmdU4$A zjKPq!j?8}K_x|EG;`w+Rip=dY^fXwE;!&I=Z=4?VNX>Ui$SbBXkzs=!ma zS_vhW!gczsC&zq69-4NaxSI?{2It2k(2N&xe*E(r!rGswqd>33X#RAR-8o6kS4e8X z8;FQwG5(O-JRS{9e6txk0QP*+hy}{M4irj0{7lQrZIC9XZ|XUrgKvKlqy3=lh&cXO z%Yo{~=imJ18hGSt?N|7({6_~V^8f7!@EbUqIsc6#|2M@@hN86193O%gcAH%)8!@%~ z9AA!^ErHW&sxO-#Wlorcm>+P&=!$gW;6MY356CCtkpwIRob8b)!KOM|shkk8rJJ$y z;l$A7@4q|}F7nGc7g#JeMYv^&qat*zs zv8UY+rgOi7De<9{Q%M#ZERmfb)SOQnm$0Nyajkif%Z}UI9})tqK?ke3^Ne>@EZH6^ zNfHb4K16i*aoyO4!PC|5lThbCn2irGppe6M_4CZoP9HTr&U~FOhwF_w%W94(nru-} zJ%Hxx_QV8X;JDvlOPp7OKC63ek47s1GE+68!4EcLiWSX-T7QP_@VS)HN_-&1c+oot zHK{UWfE)sAC3V(KE8Yc!8pTU1>82Lv7-gI2)+25oQk4uWGI%HT^1EQl|Q5?@Y9J4o?m0 z%{8qo=`8ww7j2|Z%oSi%#u^s8L5IdMy4)& zwXFKv9`u4L5i0u!Pu?@+0zb{8x2AEkIa;K88f8|j{BS$CO|4Wbu zDae1NQ-se(ZubMOH3bxj9)G!NoZ+6lo@f8$WF->ccAF^BN<5E=OT6nQDG+4*cc8C% zXXgpi{r&Dq87Ubr#)oMeUriuGvfntDGXuj(s4_js4ORr9sZ{DK)SPM#cZ0&=0$c{- z8j(08VfqA`vPPBzO~Q1-&Xl|$qk>q1-5is03~BT?Ht}})BM=paO6J+G#gZ*`?WzpI zI`x(>egj8ZGf%9O_pihJIGhE5V2w>QZB+=r+7@dPgO5|7aAS z&yje9AJ`%$s|CWtQhm}lWPp!{_fnj_vsZws!Ge&>Z^9OdDmbA6k4!VwC)?pYY5jS1 z$ANK+H_6K_;krFF{qBN}rD_E7>-_zSkhGpI3sUFO4p%gorNk0M)n?8jl*HZ8h~q&V z#hjUwhA-8)C|E69nLi+!sa@`*^t%)3DOa46-xJWD!0)mS_BR^|RS;_$H zQNd*9M?AFy-|Ap$4{J3io5~F3x}N?qMFi#IPINj!Se+p&o#>xp_q)zl9O_^$>0H*V zBbx1Bmlk@6vLmq2|MCv-irujO z%059;heOV|af#81AOdOMgZhDfBDe@A$jOo9L*zG}b{I7F5Hu7tz;dCW|KOKz*#>eY z?>~1x2qZ848R0U>xRuGh$?N0u0d|cNWu>;P6%g9ST)vYD^JFFQWXaVnh|G=bg-j#( za2`lT(YI)tDy(;!0r@B0Cu{&*cJT@GH#)%rJjQCr0`-R?V3f{#2y;G(GjX??}Oy zZf}>^I}9vpaiU(SUgdgYn-uDX7d#m7De@ppfy05pSuSR84w43^uE%AnG24v^SUaY( z)UiVPXgTQ6A%#TGFKdyA=OAsVr{|qryNL&vs0>ZwbIB_sS~|bP98dr?8&hcOqmm1+_fbP%`@}oC1n}<8kB|!!g$#dW)LbhT20q z6j$&x9B*iUTZGb$pkzgj_MGcJqD&%$bPfGKoV{a^En2cITvfYl+qTVJwr$(CZQHiZ zUAAr8+U2VK>YVP|H@+M1cJ%uq){0oM{?3^>M`n(Z19BiWz_g4GXUO56cFJJhN1A;$ z0ov8h|GXiCw!&yhD_Xvc0g~nMfcYX_#Za6DjlPH0d`)xK07c_kPjkl8qlfY&HS`^7 zV9CHu)Y0YjvV0vLZuDFUUe3 z{y_NMPg0hON?hazPs5MhdJRNdgHyfI`y4&E~aeh}TfQpKmxVf!S+_5yqKZ(RN2H9nfvOCPy? zsT|XD$7iwQjulM!&Cg3AMASEXN?jDw5e1y^gCHKZcXi?ff4Ta#R2fy@i1bZz_j`c} z=GaE&)?UH#Nd1{IH$k-=>|{B4VTskWtR<{;v@vovS@n0uZ6Ol>kf7#2#I?#L#zoTm zRp}GOJ`)`h3vq#_Xz|zs*zQ-XL?@H9;jzt1%K-v$SMU>HlBN+G03E!q&y$8OmV2kAJfeAZT2c)xM9!uw`4 z*xd)+8cCc?QoC6C(M;MGX6!lWI?TECOn36wXojlRZoR_3ji&_0o%DL3y(|;j)HQP} z+jM*>7NK?;93;9$~IYS}Z_X(SMEmg$Ucm&_>U(@cx<_ z1>gHpR4v0QJhzjs;}v^4QtbHCY!IhPS`S7-A3BCQpKP!kdN zADc~d6>HtjzY`P8zr8o4|F?VdKcLY+ZcnzdwxXsovNjon7Pu-RdANcmq;)kxlaNN` zvK+NE{~UO;UUm#!chHaav~7Aevdp_aQf4Q_hQ0Nkdx_C z5l6n%+9XhM3FguKYM>a3wQUm3Re0nvRLS#OK-Wh!6GHKKxNmdU2C*3bFIG;om*08v zW@uWrZz5hpR1c@gy_8eTAVJ4i8%hIP{TrPYF^!#s*=Sq9S=2PAL z5`W_5Y?Itpp^?Zn3!X;`$hdXvJi3$!*w69{~^{k1dV`zx?L_VTD*w)N+F zQ$vR0x$K$EkvGH^_g#dL2i=$$*`IM-e|%s5)G%y$3qpFs90YfD^SAGv^6VisI1sv7 zJF2R>#l>wHS;?_|J!KAQz$kk4%*VB!=Qm|+19tWSOTnK>Kxw;> zo=K~ts#+&oB1{EOQ{LI+9Hgk1vy4S&G}&5(>`sf{LGadO+FG7O7{k}-% zNu6n542Uz73q}&DmdR+IkJmiL9YiaMYP>g*dqxX-(mSETrcp-NlkNU}6Vk;fuCW9< zmM+qwLDaaNc_vy^bMN9)`>`Q&*}Hmzn16?}a8I&N))~Pzr1WXmqg43$;GjVUdHdbA~Uz5A~3pL>|hfyME^dBfFXp$swnJ z;NjeT7ZroA2fK7!Ud4fX_bkK)&kcpw?O4ktm;ZvZAe)}u-Az6XbP&I_P8%Eg-Yh-4 z*7}5(gfu8LWRuG2fqFbYf&_B4f)YUF@_H+!VDL0(x$`(G0;?WUi^D58ND$@Xz4bfP zU&V$>F}A+ccY5ake%k-{GvoRX&P?f_he!PP4|jeWL$m){MKz^*;fidE@uxde+N2Rf zKny%cV7{2xcujg$&WtKZK@dP1OTc1YPa_h?g>21#EuEbS3h<_RyIH3~`PN66X0bpT zC>qG3Si^SY04#O{&gbq%F4xq=#g#QR(O8x*E$d;L>xO&grk(Gm(e~qH2@AkwFIL`Z z;CLv!0PNLcJv-ixHA3UYI3(4vD=T)t+YK+Vhc5zdH?ZqHE(q@7jtP+$Q5fkSfqP;m z_WqthhZlE<>{i_51)0e!BX#EX!W8Qkn|H-;<^Zb8OTPcX>w^SBH-<0-&FvNP;!zm_ zt>AP}*yJT6sJ5}!R;V6lCu(2QtpqUor6uS_;w8tAJ4IRgLhD!jbOvfSEM|xW%xQ#m zWeWiC`IQ%EwMVUi9R7J(i=&crgNcaVNSwKJvCI+7l(@RPNwXpcQ{j0zQA&d1OvbW# zY*VaX+MF=zY760xntvF31{qj1Q$&ri2(pM{m_(lWX_AA8_z{k@ER1=}z-ZxmQa-!8 zBno@m>BE;F@f1F|;GE)TOZJR3SB80%Wl0UY#qq6_hBMq7EWF;9PxL7qICi9}l^zJ{ z6*?L?+-!dDI>D4&=tFyA1xrcR;#raVs1(&(n}PYqHC{+|l?W>zI)JKaiL&HGd5B;L zJ2tm;v!dZw2@|?ZgHVn_c74K5=RTDv_33MvgwIT(A+W`Mm5ZcGJ=AQ-8TlpUcqkb; zUPMhJiI=V(PCo8R7WHddLB9v&Wpu5@PkL6~+>^SLGlAQ{FgPVJ^13xu+gY*9cJpkz zz&hua^wxQ!-=CHHv1zc`W)V93WG`{c6m`6Dfq_NA`)&1M7Q*R3nO;1gDRYDhgqAby zYsz^I=_G0}bV|v2uDc8~$l44g3L`Q(3JWqj;~2~m?)h&qWfQM722(}F>xVWQ@=;Bh zCz9}ESn5Ngr|0_}PKj2oA2O0@3nFo4+g!%TW{r#H3l0&q%hL<46Z^|)QDDq?6B@_r zHFG^=sdw;Ww%RFaANwUt!ZpSiB7kF~7aiW0O`5J++7fx2Dz${QLJ+2bC$oYPg;8ln z?`1247_(bWPft&fmzD7L;)eQ50!-Z`ZQ9V!r51Z%vX1P`jghJEZfnvlF&$HTVZ+(( z(tQcp(6olwBgRH~7XF{78K3?J`)PlcizMY{9VGsHka}Ce7JG9KQQUl4->(%l_0!H2q7;AP(bbKfejJ1+)nK1Z9s2EMw00-QZ zhSBL9(^BoEhn>49_4$YCULNS|10y;jo+uAUZRU*Clw+K)_eVb@hHLMzFhcFxEb`vb zPgyX24O#mOqL3M>OA-_+MP{Epp<>h?P!@j4^xqhF0V?<7ox)Z~@cCC)9=i~*Omrsp zK0#i@iO^=2wq9BlG zPUhY_BA7}~j3`+sZqBkf;BH_2F%)m~;FQmNMO4C4HO<~f&w^$I3iddBkEEdRFhbZ- zK8_A_pztuS*nwvbYaCzHu*W-YO}$+xFH025-1^qtw8^aMJkI^6WGAqgyT&KZ<`(mV ze@kk?jX~>mS8dJsbw5+@F?;h2B^G)j#*{^Zw2^!VO4qz8zU(*@LS#4pD72^8xS=0? zwMAj3V0b^qHd~Vp2Kg+`1i~Z8Z9E(M+;Ye z_ipYTCbJ!g`380^Gg(L~&lG8A0N9N5P?i~-BsNW$aGE3hshs)>Q0F)I^;X zLanj1$q|IYk#tn!3thHcm1A`LAr5R4kEZ**h54qtEt{4f4tiPZo|M>5(jV1h99CUnA;3HvwDG!U?d8OE>1lp^g8jO6sJzG(2Pnonm` zp!ZG*^#_FYPsT{126kPwM@;++w^r(fcY#>vO0?s|aR$PTX0WeG*|gtoeo107cf`MQ zt>3|2_VQ_;K?qouvf~Yadq*ok;|*|WMI#DFBcf{;%&Ey2A~+B)w9??S#_xiYoYXHt zt@1doswCh!pM3>K6JpkNRg;%SYpK(c4HmZKiW?>|TeWxjO@|A)HkjgSMh=@x#W9vj-E&^U?7a|P33FiziT-@gODeSsh|3x3 zylcA2L^{NJa@Ty0jx3Yj*6Muhbl18X+GBA!DjYs&M@3D{qs3rQ41 zGM!a?^c-wG2(lFo)n&2CCbr49Cx7L%u%t~i7x$kyN~b_nBkP!>^dRRg_Y219`EXuT zBV`a~>>pJK_h@)Zh^dknGzb`{`(XYZJw&?^UABr;_6Q1BM*##=n$i^R9wd?BZoMy| zp2>d2bTRrmNmPU&nnhWY*&sk|!=wtNj&S#~mAz*w3|x|nf*{qYj!@7thK@=xj8h+y zaqC)_cwU6Gtvl)Y5C2p~FnmUkL($^LVEm@LmD z`GX%mJ38F7VD3d(EsPMc$$qY1Boj#nocOpci?y6JmB4285~tI%?}G)(*N;yk>w!ud zU+r=_(EVW>>+k(s0JQo$fjDq3JgbIsDeLN#rYE1Mff{sE(gDv31c4q=%X@W|bi)Re z=-5K-E)D^*^W**kx)+NOWQx)cD_4tAFYGJx_FiMN@UmR_ZJxu6Volu0B2u&Y_1)YP zNrQV^mm&Pdlu9xtt&$*@&uR4^Y%8&G1nCDZ@$ppoqFNsy2M8GFKnTU{L5L+1q{75> z9$sOZ*?xU{`|)aN2%H8v)Yo8;k+*l$f`rO~eD_@Z*h0`XU%!Y0hc}D3&<*#|v1uN& zb&JOQ_`|m$ak7%ndgKU{0+wob0$D0b7poCVQ+heAkOA^xGwYOldFg{dt$cz00+5f= z`xNZo0Fve30?42LzfR|$;QyZsO{_SrAb4 zk@rA5kqf45nYc$4n$EoTIA)$b>*_uM zK<|k|+ZVw+4#Uqj(1rLwFA@gzk#ic-+mnfy3^b$nmH9LLIDn29HDhW)|A{#erqp)E_Xg-kTSt(HM2>A7)7p?>`%_@SJ`A- z>af$I;9j6gl}JIZF{Oo+4e#&Dt~Fy6QbD!|*EEiI9lx%??_FDL4};n~$`pWLql;u! zHC8p6UxVoxnrcW$Nl(rpG$TD@6ZLmkxiSE!zUU^g>P~F}=2167uMUy^fdCww$QELyM#Ibw)A&pjo#|J}+h??U~e6t!r8mR(?!R62kVU_a`j2Nd%x3ff?)AuAIK+>1cGA1vUL|a} zyU|`J>{x~Ydil>FCTQ)hvQXZW5{DK9XlJK{66+0lEpbseU5j5VsI8I5QA0d8dFnkO zco`X@Ivy_y=H7A{G1MnkBoUm{hLU)TuPm5Ti<|aU5g_idN^{A$bjWmb{u&)hzsUefL}QmhJs8Cq6ZW=o z>SDhxL^Ozc8gQkXhM!?(qfir|Q{(Wdu0D|X7vmr;X=eqEzgzGp@YJu;nYHb$xe%*sLN|;?8qTzdkvX#s6H{j+gCKd5zX(oR>S9gq7Y}C$K&;NeuT*Uh;w79PFNUpxa zs`dXP1^b`%^nX&D$=QC7+?~wzt@s@s%}s6oD!2b@LX@p!ExRCx?9+k9Dq(Nv2SiD4 z)Bt0IkAGi6NH+{T%Pt@QjyK+BMWQXa4*MbMvm`VdcdClyn{?N{go{Zn=d1GrMi22u13^#)wvcikJ#t?i_<+(OEzE#1;silTn4y|53Im0XzOn#?t}rc9 zk2e4Wq`I_&W3rVHnV2=^w<3?}xeCMc-mV%lH>i-YUvcWihx-ge72zjMWQeg_Epk*5 z)?T#VWK6(pgtu77YKVS#wZ!XB_Ki3HGdw*sCr43N==NF+L7g!>l%M`j+YLdD2InUo<-T$PTVMz1FGN?^2{fWf`qhGW=CojNLh; zgd-#|XiX+4QX3>YCASh!`60s8=BGae1j#y>odTb6*CpHEA@_f)3*BlAU{N#mSEP}a z<|lC#Wm?n22!}@-1LOBs)g$*4`3!>F2@^@C|vLE#~Y?%{QOo8 zQ3eH+bM2~S5^a<#+tO{RWehu*T+fqNDbmw)vgr5%dFDWla~4msaQN1y`;o#BXU^Uj zryQz8zKVbjo=ZWlLyTb^At_qD;uVd5me^@`lhgP4{>=Il^pl&G3wU>uG@oR!73=vG zkZ0Ed77&2N_S*7bRnPYk$ZQ3XF zz8QAY`v-OpC>){s54u@OZ=7(sQ?&k0e({7q;VB*kJy=3GD?)-RLaD?FV_pfszOxx& zR$hHdgdUm6E&fwMm`~A$Xl9*$DLXLsOV^!9&lNTDnBfpg$+sQpwh4bGDhhmr*hx8H z=MwjM2euTiMe2eWbj?#zs}<&ubh;K_;Rm9i#e!LWf2P&A2M&%~iN-xeB9rY9)hN9g z#K}JX%{O9lNHzQ`s!sl`7MbHec(DF2QT5LrwW7A=UnG{sEu#a>4a~~rT8rl5@Bm2) zUSUW?K6n8XO`!b+BhM|F)ybr_wfWp)*&fxgGA4Qmj8R!b3CU2*@_ zX|~R^$<`iyR!u3O)jjy|P*4^QugbB-aots~jcf~kIVn$F_i-Z(!aH@a?>l2($%A!f zrIYRd3pO{`>+bc#gaZ{fvuk57J(s~kb_T!ap?`n{H}X#aKj}*SXytKG5DE2z7s$$7 z1+YuHqTK-AEia#8axcP}{vv z`71!Q!i?dhXfJP!TL`BTGmjl2BE36 zg5g8QL_cY#4h%Ra1&_is9S1+R8Wu)qBP2%%Y#7y?x(B%e&17`EudtE9T=Jg6YF%i) zuuRJuZ5v69gfOb!E|I(d73$;iWbpQpgM?v7ThmyC4bh<|; zz;@RHLZ_E)=$5Abxd3iQMQ*Fz@V&Pr`x6JNgYD{KXREW^R##TEHsJpH$2xzt`zEu& z6wkN!Bghx8A$v{wfw*?x`Q!rY1yjjTXeO7T2IUf}jc%fCZMIj}aXdpqTB7%KVf*Vf zIr-fFT~K+t2w{#womp%orT6^i9<{si_HG*j*16Om+_|_SR@B)iat5IiBrMIu+|))o z`h4)bh}I4-FJ~Fq`w$68@>X~mD%Na5h>?M{hX_b+#w5{wYwU^oa8V;Ljp}^dC1la^JT?~fCXPa~#4tQdo0TLiSj8 zJ0jY)IG9oZ~#oLXd z&r8iGai+c*7p7-cwzer`qNTE@i-LATB#vVge08{kL@S9(q_C;H0SY2@sw9ddGzJW7 zsk;YPh9<0|OOv~*F-X&JRBbR4y8BCmEuU;wW>|1xgexU>S_*q&(3u>n7?aClbkdkM zT8qp;To)5*P<9LRa^Mp-?mK$cFqk;XtCcwXLcm)uyD+rDrg{J#pr({if9qvzIrm6t z3TyqzD$$wjQWxGb4b849>F%3uHP|V@WGWe>qTMK8sn0s3v{FEn)nw!VQJRK(qNHa> zj%}cGYi5?hsndy?g3>|Flw<;2^`MsIzcy4*s2<(aMuXW%l|x3HOXMAXXQZ@a71pY5 zZ4=!gT({RqKV@B+g1NUK66`Ok>;-mFB16&u6D}mRvQOEPpBcZ$$xb*f>;A28B#viJ zPou3CA?KAi!@%y`FX5$r(wgi*y4!;`$uxwBo}`@T!a_Qz65#bx9eTd?#`DSB4}FP; z<5Rgq`cN90xphSGUcDHgEBR^kgbYHnqB9??3|sa>f=IbuFCz@bTK0nCQ@u}ut!&Si zKbO8S!iHhWA5oh!3Q|cW;N1vz}qVMrB8jIx?5*lEQq#iPT_U+<0l={_e%h zV04u(_xV0`p#Rd+3F3=vnTbf>ukF%3ZGQScG=Lv#!S8EM5#KH zx$vCrzTawFsZZ70O+nr|EKVXyi7s%IK02^tl5cs>V7%pBE4b6!Mut?R+O?q^<8F~P z`npFx!txW#Q3#1vK3TrU@7D_6!~FA86>ZzvKGj%W=N&{L5*@{ z@@5rdW%5=P6y-F-YRXDE5*0N1=PKZ~bi=fMe)6HTiF!=7JWxsbNKJ;=+Y4|@-WEf^ z>kF+*tjhlQlJC_Eda7*{1~fS?eo5j9 zpfB}dKT#$1p{gxg=P~#;@jS*t^C#d)Mksfz!op6 zo`4V25}f`eVmUOq)p zk8}`y7_QVa?2M@2Qp3p_=(9WE!t9lNhnh)ae8&epH(qk9L&FmduDm-(<{ zu~eYcxxqCLvV4*D2%hzxn>C~~V z67VS{zhT2XqQ=@3g-)7Sq7cjSAR&(rTONs%lP|eW+;fc>}y+PXZ%N zCI)2H>Jd~62wu4~Z43tG!hOy9?IpX3DHhQSfTu2iW=;Ui#{g!NKyvsxkgR?q98f1L zS%P7&T~z>Na)#9dAZ-DoPoq;o&ch^|_fIlmtui=@bvfnCC;`qvE@AuCkGTPhegiE# zSj25k6`GG1)^7udCBn_y5W_^9o;)$@ewoD=z4M|(#wRbhsye9a`!uAsySTd4tcj?Z z;tc839eyQNyzg3G5lk=G2irel-rufzsgZm-UeB3*B04^lj?s%JWkQ;A2d4vLaOj0=qCExFn2B9cqw;CDeuzOY#Jz0Mo zRL&w+Cf$eRzIz6e$(LnCH+fuirK+NMeBX35Wwf0BV@_LVdCAJ!@@Y2zVx2X0Dm%2~OHft`?SrN*bioTBl{Vr}%nFlVeH&7nb8rCpA*m`r==EySN zox}3@f>ekD#m0Q_!}!gy6UObpiO`eh)!^X*&`r%&m7~k9%3$g27W7Z2)d*8J9i+F! zAy=5&>k>hzCZZ3|%NK|RqZ3!I$n_Jes*c)8bV{XZY#|iTj*8R;o2_lY7={2s?jV(V zehmcE`YREgjka^PCR!R3XoREU#92z8F^7B#%xvI4{b@F@453Q8^Rsi0U)x4y!5cH6F-?<;YiF{U_E87(06CPXm56;n=S8geMZtO9)C5 zMp1*T`*c(l4@1YOD@JkaAUpLzjRoWINLgPYup;2;YTflr6R*(d?mqRAHc1t&@j4KB z73Jc~%s+Wh+L~r}{4JeSb;-=L9xcuX!nY0UgmxGz5O3tXPdT>pT0*e>XWGQ8_s8@% z`>jVOa%#Zu6XRvQXdIVUT13w|#@? zeuesPx^@=ne4Lupai&-7R{GyUrju2eIut#wUfBF&M#}{i*59?27V7#wC#;lR(n(rywGDXq^?ScAhyk@GLxR zq4-Y$3JM)b7t9nHm)6ckH%s@sDPzFkd$R%0H|hGH$BsTJdGYt*#wS7#@BOrJmFxwELVuVIiW>Ea9^Kai zNarM##zNhb0lg6O7VqZ+Xxy2%XDQub=s&JmH0f&%4FFOj1x^?(5xFT188+N1!KfGr z*pVt{!RFzZ0-xUjOz%5V=ES{opyZMGHbrA(2EBzkD|b$tca3)Y)W zOQFww2`yi$$ngx?GBBmip;+ZUxeNMIGGX$hNs>;M`LI`;A{u(kM`6f6_~ncfKUz7N zju#OlFnLg8-M4!p$^U3K>N{^igTc`OSs#`Tk;ON?#@xQxp}q za=c>~y}{U=naHp8S<%T;KO0=JAyW`_8?_|3r_sJwh&HZV-K(UlGEu}38FSJKhN zFd_o}bWP`!AkE^Rb!X1CRLo2RhFmoQQG66bG}B<`e1#>h&^<}@iSLvcw-t;TL!aWjMXo27lJnA^GZ;mZw+%a zZ%&kNicCCi#Plv4Lt$A-nn=M!D$JJ2dm~_k6ZLXTv_;udp?CMC)gOkI9w)^0X{uR9 zPH<#jBYhH_U`DudMHPRZfpAU4l!(Qk7#I#GSvVvDV@B%iEYJ$`qH z%^>cl;IjUSe9<>$Cso@I^}HnhWCfnfy^DY;Q7i`0+2^!k zzG~5En!bUnW)EOj3Iq6ABfeuMp2LDM$*#bE+@Sy?yK^G6F~EI8RXMN*lGt6i*R3CBjO$UTG><#PZOA3LX>^sH=C$P`@;W7e-ZQOkK6Q zvy5`Iubyx;*&0hTOE`cZvs}`SGgl<7mUg~TNZt$bu}cr;&FWrz*UNi-LAaTC(?d42O!Plq~A~D&Qa8~s#r}c0l zPPAiPUu@cFY{rt5iv|VYT^v&a_GH+htc$t1;JMlII_W)hjI+%$>*~p}ZN0xf=_cTY zAi?Gcj<9>Dc&$WS?xM5J{dr`r1S8a6nxE5`1*zqZx-?*l zfSw~HUF40p-Zs&)tbsI)Ei~FNV6bL#kOx=Wq((bZIU4p3NyE(nF1c6Ij;LRx2M`FG zD6<`GjsJm8IjvU{5K7%+P&@wCu-s6%rXj*!gE)os701wi}(Oacx7rAkCYJB`)-}$3^|>*4c7B2DP+cT<;3)b#e;C} z^spLEa9N@ZL!iY#BG6)#eFR6AswnPS4_hQrKr?kmHcP*{d1U+8n)OptwD1=PtrFH} z%t|y;8lTf-ZPv$5BTg{Ds_-&YGVt--s>C*z8gwQNV{)8t;`LCSvWid#X_p_pulAD)Knp z(FnjsrLkQzXceS^#Xf*)qUeAhSX+KJPS|C{?LWlau|ar zI{@WBsByr%M;uMJ)hN)1-@d3f!EH37QxtViOPFD^K>v(;filnUwG5uH3?8TqA&?@jo|4vyxv#GR3M%Q+y%Hk1w`(m>T@~n|n)a9zj0yco8~hd(POlEUPGEXX28zWk zb>1QldBd^z*H*yR>iujJ>Z?r2i3?)afMUX{yy}4bl5KPc=l+hx@0<^n`d`kPf(KV_ zpr`bx?bG+JK+i z1KCISsCg9JYFs4ESU%gLrH&f{B}}W*5Ysnk9=UFll6B%)_9x1|{+=>MZfZqvd{^y4 zSpN>7ivI_t!M^~iqMfm!xrw=2 z$%@g6v`Bo)0cP$PJzOaH4?nU7cS?IyE@ zLr4!K^O32GN)lM-R-=vn-;V{2)YRTGZeKyhZaS14wfM|^;vi%TB{Yo1`w4a@Q6!N} z+2&-XY7Zfn@^hJTTR}3>sezp;aTkl$JdxW z3TVgJyj;qE98)j6B+Is zw~{qtBny?p9r>wV3-e^y6Nnt&E0#?ns>Hrsjl*#?t3uR6vR4{=BtfCg%Y(?n*r+=U zgbhwnRH7i#&`_LMh}&l#LvF-g`DZ4n+I*1LH_wwZuCKe1n>zP}!?(jDAlnP2nX_v8 zsndM)$8Q}$rk9yrnRiR1XJp5exGeUU)>X8j*^cNtO6d%NvE0D^e9l=rUqRlFv@P7s zU>dett6r-V54oc4CEY^BXhot@b!UX`os021%$(;W39(^6yQ2h%FYTkM$yTozxehjkqk(3ovHN2|_@hv%ru&*~W{4>`i1Mqpa8dmTKHv!jmraGzrv zowK0OF}LC4#Yn2H)DdK6Dk%K%&*z3Ax)1O9aqSlZfx&AJ%?3bq+e?0s!a{QRd-3*h zX6^U(XHWZoYPx@5tFo6_krZimsXwfk1{NLAtg&#uZW}M=^wK zLZz>GaNnlv@H%sMa2ms6oa1r3i6^}k0(cEH`pDSJpZ!nWs4NunD{KSL$Q(M*2hiS^ zKbzc_9-ZIJst9ExgNW-Mx98Rpxt9#Fl85d-5QD(I@-b$861%TvyGoLpf4M(;KrO<$ zP+b%%Nct~$pL@`AF|)cT6%ZntyA;NC)dEQV>t#p$cjPMh2_hhHy zEOo?e6Dzv6JjMz<2;x^GyqyLgXgOBn#UWINunA-=(qVX~@ylVLn%f*joXG%-dLJb$PeX5J}bsd#*t7Xy^|0SlS z!OE4kOEiORUJI*E>jkVQ($v-YD}1K`N@t9cHk>${^g2dDc0Nd~*#z~~xWb^}dh@OP zl|q|wgX@j&UlaFRQh(&3Z@D1Pw;}rfUflEkzvNB5HtfPfx z`NdkZxubkv;gMg8LMdOi4^k7TSvHY-4ad*E*_4z#B?|^Q)#2_Z{sh-aewBRS%b z%3174CCT_;Mg{@0pe`vz5;w%7!uSM%)-q*ksnAL-A@dr~fYYY{$JcGUsn7s+A`G(4 zLye}4D?q+IeVZc44MrxRD!%_VZ)1@GvX)3VI5hN}>AAl3GRM9Bo>&;}+dFtb!QvTZ5xs>GQd|b^+!ce6YTuRX z%jseP&Y{E?HoQPjgAt=&Gqd2^GK1MQ`1ux}N@xRvcF0o7LLw8yHqr#|TmGjQ_hz`V z=qZ2HIxAWJ{4~@&N_S4zCt`ooEKI)}+4_InEu{?^Bza`* z1sN`SrE8E#S7Gak;s7mV_;&K)q5^~fv>aGF>dZl>;jWNn5U+ue^3pxh5+ z=WN*WpW^h+23lj})bbd{k5$#@AH|rN1qMr9m*L zAxeqSx)Od-ru)!(8-0QGqh%&wQsQ9F1VKpnJ2<#2Lzj`ZRJ%J@`#SFEuEkZZn?vs0 zu+Fx4l`X{6W)ExsKo}BkDuVt^hfI5JX z+6@UF1GX_3?{@-XN@JndT1dl0e|c_EGAKB^ufs(**v21GZJhD2VSqo9lrCWkg;jpw zkK*&|)dcrCLVX0wT0=rEfbP%`d>(tBd`Ruz)0I(OP^p|)beaIge1?O1CthB5f`S#l zLq2&iYO_rCCb%)EOWYU(X z{#1+`Lv~naPitIi7;L1fC+RofUD9ueWz$9B+f(CBx6gOeoy>@F+QC;FGFvbf7s3HA z!zM*`d6kd29Oy<@ds~FF0r4yY>5=$h^DO?T7vZcUiHAug%1nK@8AWfLHClQ?fhT^a z2dgFp>o6@sL>13;wT`awkH7ojYfW%(+w0jU`7tZOB_`>wQVp`P;Q|WQXq$=&m+4 z$X$6*F+OfE!l0?XQ9bssbXC|@MGy;vFRn-?dc%tVbUXd>ofl5l&x*KCQRGV zcSC4@3}w|T^$@UnD|HSzg^U6;#S4-Jw~9lS9A2`?wUP?71+EjJ%2&pn=7l>L&_7x7 zyU<(e$|qFK5;>U{Kz;-XEH2@30!?N+$ZuB(77WxTFpX)HD`5f=cq-u@!51wB#g~X| zQpuVW%B}6?iAK^4$&jVdZ#;BH)D8=zU}#bc^b1DIDwBk*Xt1vC%A*)>JX`wUNMd8; zRq$!ME<-$!;>u}+F5S2_2^lvl$kB_R46$NXOfOMhuY^t)Hg!d6By4g$YF^>{a;!>( zo^($Z_D=;^ly=h9WylyI7;xCC3XRG%ET(3ddvab~`5&BoO=AN;;)xIU{GPQ#SgB^{ zy+HQ$rI#SJxhTGAr`HHlWhJ^lpK2K?Ch@5I{~9j zP2^^tfhzJR7G=p#D&n`@EQQ8Vt@%#0;aYQwK1GM5TZS3eYmSN9D050tB~8Gg@d{J~ z%t&qBvuPN6>MLgGg0++6av^?eQ~(rJ>T7N1_(}!PaL)M%#os}{yx062OG=e0wdm-j zJd|dXQ)e*!opw<{M9w;$6}=qIu=^~Tf5~*RcKHMN@l*Lz`K)X-JM~X{Q~kI6vCN5mYx3HYJM3IE0XTV1yKfdGfTS{pa*?0U222e3QvS& zfI?=S*_iP&;hHgiM*5+$2%#fBd)4>v<-5GU;2cXtVNAMwlEbtI2U1l*N2z%BMAEkq|2l%FRKThzWk#^vdQ<_XASmmV%e zFG022a4(aJv;xJO7aupnydjci?xe6`I_!jNq&XQZk3CJQd{1-2AVY>jK*cG>s5Q>m za$}))ONfTfWtYECyvv7bXA)1Kh|sS<`2{eBmVtrfU>32c?G)v8Pxlv&byCCAP5lNL z2N3@b$Ey6lq(goyD`{hEgZ~!BI}l47JDJ%U{VPwZ{2$uhG03)UTNhku+qP}nwy{>) zwpZG=ZQHhO+cs8a<=*$4`zk8l-E}HnRK%En=C3(>8@;#QTKk}9>GUsu`bt#`B`g(m zZ`n9wAxzrCr3ZZ-E$UrDfUw0RU19PN7-Rs2p7lD8cxvkQ)J>Sd?mfI43$FZ`wb{2( zxKD7Ouzd$s0L3^2QS2ZGx9#K3?&Hmm*JrwJ0Ce9}VKiRxJS;rcTSu|}P2wRtxqdGg zI7HW!fS_n*Y2usOAPE?yushpf}%0mL3R-!wnZY)&Pv6^9Z(S>SO z*a=@H6_{8_kVa*)n@x)=OU7=zx$G@)a4rny!7EAy7 zfKFAOXYu^O)8C&#+nMSP1antAq+n~9pmM0ClGfK>SDZ$eq@lDbZh|SRGdP!0;)V*r z$#=3Tf8U@JiIJrrQZg$vhO$-U7H%u|bWJqJcf*u7?H0a6xr{2@s+&8Y!n{lK>DwmA zT(ygX&#hCe>YZ#0#B)HA>7yVkLHqm_P!jSPwt_pwGv4Xfw3f9ZHh(^L>*=bT0?T}z zd+5SE4fE;S)SGO2*n9%A=KUM>9ijFFO4~K?Y0=lqu>+Q+_+_35_sYyY=Lzeh+SMz* z%TJdjen~KUu0UIm&c>_b(Eu69HR8F?bjUU;g?NiZ6uwZllRHw8SI9E}u|EL=Kk`;N z9~Zj5gpsnuqy+QPGzA1dmvBw!xW1ozNa(F{M!(FDqpB39fmg9|6!}m&;nrJ}Ity<; z9`^ABWk3&SuOS=u)lL%=s{GBH+$~+bW=p}j4|=QUlG>$#18=ae@D{L+rl>?vt7juF8ujpEit4bkex+b-Thz2!sK|=<*AQ;E3XW>iEj= z3JuQEx_vzdG8xkF6{zR5O|(`zc}OYn@HbweMZml3xzn>B*@R z!umble_rg5yWX#Oo)v( z8do#JiQoSs!0qb~JQ@DY{0@Bqr;m*bvG;?QYBFDdJ7^e&n*dI3ti2f8Q`Vc0?mV0I zCc@X`@D(bsq{o5rrGwjB0o8l7C)po-pDNH5fNre48q<{`MAA(swAk>#V$M^?*oP>@ zW>4$x#mcM;X~5;Sj(MBu<`4cEuqci$MGxG;b1=wD?8j#dJc+CNK&SZ*8K}4B?`#kT zJU+IYaKI175c?0xke~Z(px#v7-AFpPtT*rYNqca9Um`$0%r|5}KB~X{S~r?12j$aTy}*oM~C#1La?smw^0MD#n8{JLpSTTciBPpS4or`lvXotZU~PZAS(oOEuT zsZ^Vn&Lqq!ppXHf@-u40^+XrJkES(?ceKU?0&l6lSG}zH+>@%7)!sXTXpm~pDt4t! zoRD(*Du=F>K~e{6EXgUB^BjMNYUftF1%^oK+0dpf)inlU;Rrlh!GcEH$#GWl49YE%)k?l1sbVrXz*D%iTYMw8uZIu_riffQwG+;o9Op`FRtNYkl9 zOe8Ec4*I@XxH4tp-bGxDAR&lO&uO6Idv=z!&ndfxyER868$>0mwdVdwTBW^n!D{*| zlepzzuZ+4pGK-JKP=--pvTlPxizJiEN^(qjMW=cuyD&}tg=W^e;b{uS96QxonwW51 zk=TY8I@|L;qm2A3X&#F(&%bY~+4z}gd==Qf$Ui-lL_}*@OHRvNs!~W`!D~#Kb}M56 z`Uo#Dl*5=E!&E=Y^RQ#}(Z?dx5m-X)Tw&-dxBTcHFtHKGg~x;oIz{ z{#Ytde>Ktm?3UWX5s)kl``J;a)M!9Mc|iZd4818K#aI7fFa6 zS;rN4kOuo!g2oV=gf@Sg7|!TS8*@1}ZF?`nsy}&-27{^a^U5|5`OY@HsaQA(PK^H# z7Vt2ZH3|9=Y=6J4E}aEuH2Jp@^`=}<{dDP*GA9_~Y`Ikhg+BKxexqu$02rGt)&L>Hz_389jV z{2=L-C|)H?gn!D4@455_0cB!|zEEOXnRH@$3wm-&3v@D;d0uIhB7u@hNj}p7d+=D? zmTASNWQuuQ+QdYs0H@Ll$Fj1L3cqS#Y9UD1m2*k{{c`0}lC8Xnc)>Qtetl z0vC2;W=qK2TNGkZ8YwO7rgGw$`j&#|1$0mQ(dz$CGer8ICX*YZ4gt}KIOKwb0Zp+U-6 zVyIzZm&bbq-bJr;Cu2`!PxH3wszqibDrSG%v+wA23AMJ9ld?(XqiQ`-EzK(w(NN^{ zP!g&$!(1U%@Bvp^MWMsk-6>$jR;G|kZ>dc}^BKD_EzdhToAeAX6YrRybADfYC z&`b6lBgh7k_X$qnn@FMx%J)Gt2y$K~8}f-(JKCyPGPhax__jFS&@eT9bzJdS!2;v$ zdRH=pq3hbOjElgA#*#4;0{+`veNqu3R zgimU@iI?WQjrh@DzkfacxGS^375&uZJ>12y945%V6nvTa&7dAI|Ehd7F@#wG;&=@$ zc{m&C#2$Yya-{N+m^csH8mzjA7~MYG0_&GnOVPXqz$;WGDY)UGgU(sOW)~*eC(aU?uXc=ppzH=0hNFVUz9f>%A@FrYfyQ_pA_Li!1tWD zopmfo?ymElM2}JnCc6x=NF3EDGQybc(uNgi^G)fG;}Z-M_)*T4gAYbVI}U`OA|1I2fu$;9B-jWx>$mt_;1Y7=-A!gn#t{+#nuZdgu-TL2b zFf=3fYkLN(`)CwSt=nLp461rGsQGxNR9@OhGnt(PZ|O@RvrB-0X~IMxc4NMx!`%rT zu%9dEZc+>`K!(ftV0iZkVfvj_o5yZX>83e?M@y*e1d^(0MIALbuGR$aq%p z$n#7J3kCAZoS*iVoqUJzOy=c6T>ulvChRYB_yP7Zqoif+sR={$*$K1<2JHz{thp*FbVb~`1)b)oa=|6oe#v$@&POfD$lhbjW++3OiyMt zjI1pg-99m&ZO;Q^(b$`C6fA~N5;`q%XZ?9Adr0V( zY7hO{bd-HScOUHePVYgMCfLPz=cDbFLyA13TAJzg!)c=3!Tg<}`Nfshh#R+!ueHHMJT)P{` zKb)jJipq?j!@A+)v@(Qn$Ao%PF=CqE9+d8s#3Dc zGZvVr-Rkf9?*~^cMhfrQl%$*Cj&z$dp!fF?(Z!Vgi|x_R%gOGbdz`4ML%i$lD**36 zj}_T6g+J_W(_=s>KB{(qexPz+Ibs!*v&7Q7rhjJkNXI81ZMob`W)9rxb-j|upX*?n zuspN6_H@HtA7AKiw(Clz9S~#{2&vv~!Wd54&4ayAc`NI5);-mVypp|+dB(fw+gqhx zyK;zP*n~Tdyn3})-^@Z+A6?@jT0b)`ILAiI(UWXQVQm}-YEgs(kx@xHw}+yR8TfEk zHt4)dri2=_!`7L5hS=nKu+1xuL9vMi#iJN4(R?_ibS6^Sjchfi$FE%aw;h@+d;;u-OJ|$>zGa3TC^EYV3jQY(x(wTBkOYIrAmWWP0sguJsp- z%aLstK7OvoIi^45YZR}leGgIv*ZgC?3giLbI6+qrQ1=Wyg-whfWiFu@ksPfRhQ8JS=IG#;zwa5gLNN2yvP@D zNUuN`mnNjS>Ou&hh_Iwy;SR6`Q{A$rapk5{LVe)~0K4*lCWVr5m$9hN1IzW({Ugk; zV51;vA=!~>>l>DC%<+K5*|oT8ly_U)w=K-4by*+NzIzGtbHCW4gOxU^AhN#RS5YVb zl#-Wv6Pqe(1uYaU=Pg@#Q9=)i;Q;6)40z zbC3}#y`otfVp(EkRNu9DA$VT^yc5`0QIQew270E5nM|fKMvpH49HR8FCIJ~LJQ`R1 z4g^GnKvEr&6sEz@GJ04X1@M)HC~TWug1FQqlr)6>BNFhDH8-{hE@W{|BiLkZO{~|E z+rTY#Rb{k+yo?|=1~YH0*?ZuL3tV4SPOaB!9d~jx(PB_)M4cg>j{UTDw^a)+cq{CX zhYMjrvY!rvh*ux5Ge>Nvg-gMw@EAO$)m0VpBY#U8rAD~rg1@MXVx_4V{9 z2@_*9L*jb+4uXhU>@P;_DBsMXxqrX~w%`j3lr8i=tv*t>HRUG(B~1CMhymWSmo71KkWIsyLB_rCsOe89Bw6B`jKJ&)+l zw02u9CDxdGJT}L82JW4R?@0(0O~^orm!i=T`=>)g9bn+z6-Z>F`VGH&tb$}y$QZX8 zVs=$o#z&#=0(nE4Y#uh!D|Nivmv;j7X$DRF>$V%{rCXrOJ_Ko0Z*O1T@${@gPmZWT zATK~UQ;Cp&1!jf!G;YPLyYv<*gHr%%MZc&KFQCv$0L@C3Gekr8cNYl^d`OhhQuI4c z68{({w*Meb|K0fhf6U(%du6}RG%u(?$UJ=VtWIArtbGJQ{fG=q7GK++X9d(6YsT#= zwB8A1NO0c2@r*K1^TFVP26!f>I#RuCoSnD%bhm$G?6*WQa?--;U_&SeQz|+W%U?a3 z08XiR$^pv!6mg8NJTfe@ufi~Ux5Iyw#!MXknPY%PiPCaofaE?N5z4JeX1EYWNY!Y9 zhKwar>OvN&9ZSMA8YfJ}2($z~I#n=bifBTLe}wVegs-SaY-MaZ^PGGq;GU|ocCTWd z+UDH}OI}MErHH2EyXMl^Gv6Ho{Y4r|LnL`N&TN#%&4Yf}X;sCg`ud~28`L4`oOvgQ z(6;MQ9Yj+kTT?XGh3lu{FAz??8ArXn;g8DWnUaZZxxG ztHvMiD|+iI$T>lrJKz_LihZ~SuqIEl)Uu5$eB>QZWPgQg6>GOn@;h7}{}?Ww{~%od zdk!Mwf9hlt2nY&VR#V|gv7F<7b}3z>iKCglq)ZUY#2~doFbz zdu|ZT%*twu*VQInG&(^4Xc-LneWA;!(ZQG zY2nH;`>>V1(m{HI*D*y~Ou}?3qd0wqw8}ij>-Np%LA~&DO*aA!Z-(h0IhRH0Eaxf5 za#VSl78~sOrBb@hYmg->AEVAdL+ArnV1BbD=or7rWg@>_6SyVsh3NnvaQf&k{So$9 z#jl7t)3VC^3yaUe3rx;V#G`@F!Hc@{@5_ssBu&8eBOi8HZ|inWpJ1;9E|T641bXV# z1%rcQ#eIZ!uyX+woB>Xm3QC~@B+hbdY|pTD-jkd5`=FMB>BDogVeqGd=zD{BDqQ<@ zk*K5{vEjfS1P7#;1w!C>q@w9hHwR+|@a_IMqV*$!;(y|HPXDd=6?!zG^5=Vx3H&3W zknTU&XaAkL{*}36|29Yqnzk)>5f~ir6W4|h9U5X88W0a9r35X*QWfG!BIt)sl||5< z420Ve!jO{zmPu<#IpzC${d6e096ny>`jOFF1!M%)gtMX-gXe%jA|&Qa3xtK z*lZ9fua_xssz_ngilWJK6F58Glu9}a=Q>rtf3`Y-vG z`;T+f(6n!p-H^l@T3Jz3VL*q|HC}Tf`fm;LsT_8^nKKxA{BwM#qYn#Z4Em9ovAOK& z1M~xYxnjV~I-MVk*y{K@!@mau{7E#&x6MIRPc;Y{MO(4^IFWM`MraM;S${FDHg)}- z)e20e@btfrN$vlij!A}p-8EIMzUR8g-n0-JE41*ng_VtK`fH@VtT_#hAPSPXia_KA z3r*(j)Xk%P#>uT7{oy|SHlL80wF$ErKEIhhLAzfBzWn!HuZ#(_#3AbqjP19dGLAFu zHyoxryZ^l3SN{mAT?@7LEJ(A%;dxPK$u>=ynT_J&oJ(1-LsK$muH47vU6gPX9rp4p z$`8?ialkxc@$CJTK)Hdmj#Q;`3oK)zog6MP{NQ!P^jnoxn#{m$(J5GJ$~ONb)LyUg zXsxcNd8xIgd?T?)DF$-*$h+5>1dX;*`lU<9dC|ltfq;anDH}t-#f?~y0^2aJiz0*} zJz)r=YkX)Rq$D&gY=KU>6Xgn7awAz_A^&!7oQ8pz3?eMI*tFfVN>%45&ybd@VNUCT zdR~Juu<~eEU-7ormP(g6Rc|lfD*VLr8l8b5qn>+sHz|XFVcqeU<1fwf1d;VUtmWutSD~l~WczGkhS6 z!osiN4h+Y%u;_zR;jnJ6a>I_>B2tfbJ?^)ep6Y86*^)6(;HLX-|aKJZZnYrla3P;(^XLO(CGmWmp3+o^-;2#nOZc4hN7Zq$+s}cbdHu^ydgyl zd+H=hyhLUCLFiaAnG8Wu6LnYB-Pm#s|Gh{xDK)QEdB@3`)V_seyzFR#1xJ=m`=Pr$ zD5?VEOz=FY>|TN|lj|@1U~Zh)6f$C_3i%{x?tx@XTnyI;GSIHRF=p--9#tKD>&JhE!QSN!6jFJKD)O8f}=F3r}l8C zV%e9_Ws_So3XsS|CgZYF;14xl4wYG#KT?9IVJT2;hK(4WQ?3GoC|`x0PgBx+e@(?Z zungyV5Uvk^H70|;c4qTIC)a)JXlxmvMI-0EiO$?t)#Ky_BLAc?^W#%%)*~Pk+&TV9 z#g7&}&Sqk2-ImmLnijMaaeHA$F$CgX{5XSDO_Qmr0-uv5Nm$u1{{=lY=YK(ixhaAb zw@`c!MAvht_A`fu?*%awFOhaiA)QcS2d+cL3a>JjdTJjBAH7p53As|VEya`UA8JlrIH?J09q%?7SK^Nk89dq6uPFp*F5u%gJPeP7&s5Lwa7bU)n z_HwWnpV-N6AP=;&JA`B~dT~>VxQ;Lj-+K_~z?VVzZmut+l^-lt{_QiECy(Nvq4+M> zL@vQ2=FAb#68^hD5djU;8TE;r4G62eO)7{z+dJuqBKG zGHub3oTldO_HP#l7ZNq!^+wT>odkhH!G9&BP_a2hX<$h#&j(#F_9!L-|FfT8g$H`# zPx{H~)xQ4+zcsT5;ANY-F9W$Fr#Z+s(}N&568Gp8HdZ<9&Gzo<6%9FTHI?Ads(B|MAL zJ{==~53AF*Wc&qk4_}w95W&`fugDeq+Fm$CiS$#XmSGjJd~?$`GJ5UjayRjIfAbn} zHJaUFJApzoGS@vkr(*kKADF4-=rn=x1ArZYzd-Oa9OVcbd}`735LR*(B#k5sk@qty z>#zDl_d!++DLwve@8Bnc(Q5FWk#zqtBhmf`0sQyH{a-hAH3(1SMT{@GIr1}CDIL^U_Bf1k)fC`#ab3ztYD0G&UF7shljl-e^g->%! zYva04)|YzM@4N+H-zC=*Z0`k~%~r>A=k<%v@cU6}auz$}j-?1p4i?+U4J(E<^e>n# z+I|_v4x--)EbaTbIK{Ne4a>V1*qNaNV zwL`qI{Tu8iCx%Zs_!+Xg_QzK6x?xK`>~-=n58;I#1}wxQ5zNaEUh@9n57&k0VL)|! zFw~uXMo;0O=&_sHADl4|zd&$r4K|{O5C7N#CG9}yISUtfb--rFkR@_r_K@8c;AKPp zrH_Zhdp>+Ta3<8ZOHuo1hVbQv(X}(o1fcz!?(Mb$@26r8EbKXiLjx5qu560uW$3U| zJy%(wqPwK8O(6nXg$iU*^V8LgvsrElDjwwOdZtx6ifbDZf=+LOE>YEUp~1SNgZw(` zp`?-xVTA~rMG?XERNgQq>qtW=F3qzKtz+?FsGcl7lq>@JNl50jPeP`;d zEU6{RykJp4traHc(o~#uMX9b{*HG6})T_r-tF?2AOui=NrbdJkTkovU9AbQG!6LPu zK)<)dDHJVQ;_uQfjog5o!TNZquQXDWOCD{U28EnqzbeeEz$1%UC?*iu8*0-|v~$2X zi{)g&PN3D5!b5Zn8BwncXh%|eRmGRIzm$QrFnLll;kXj#Xsb48QFZVeHJXfiT@5k` zk1dVOtVomt+Sk{S5rgGnBu!zgO71Y*DA#`tF*M=AXa@hKlf*weMgHl60B#pK_raF=$xHz8JoIRT5r+=ch$_}i3 zioy_U`s(?35?bbuyeL&?{RzG(L=88t@$$iOBv(L<6=d0gxYIa!OLHqKIv^NT0*U;} zo|1Q6T8BD4!ldkG~qs zH)+X1mm3|vuHwD-7l7;Oj^)d}>A3Wt7&X#FybYvQ!bEKpZ(lKu+eRo2R7UcH(KiMt ze7#Tys1b!qs`O_TT2 zZ{dL6RN|co%VjS(-q*wbl7=(+2t=DE?i^WnRCM_&3G3doq5Py}{359L+51WP8Tm!7 z?!H$T2l8}?Y=-hRcu)Bme5_nhD>W#^K9oQ;zMtclMV-*UM$3~DA|}Rh((8AJYPkqA z#^~=(ZDzL&t+@uinMShYXRS7KWTK;tTqxSxvEKymU8t(U?W)kbMyS_Bi<|?7Q`%7# zWMxKbRLCBF<`yg9PuA#~LeFxn)IQGKs4!|H8nvf_lD(&v$XTeX0w~$(=RmsNPeX1= zAE@T<%SO;t0hdC zqReQ{Mx+zLaOh#s*J!#n`Jn*1Lx>SZZtHV`=f2 zNi{UM;^Iu*3HIa*C8*$`NWS{EdOVLfPd{%5-f*Zs8}CJJ+KZbr8J?t- zLKl&Iq>&ZMP>j$O;Xm=;Ae+a=a5r^->ye*7!W>mg21PvA128{-{Cv-|BagOI4aZHF zRQ1>y1pWM-szKQS*~yBA$C?+N@y4O{I#Ff0cd?k^@zQ|~l(G0L#>R4w9m%a6@^F#c zSn1&KQ9;)S#Vy^=48bMZF%Wjy3U;})uTd2=8hWdPN2UObw;P9&4u?`L zN;1kS301N~QsYtMM>ipvnueUWcNL!bbWgG`pMP6q_&aA5?ehZd^VH4M%|^5qr?G2t z)lK6|nnJd~j+u14nLp2CB*9Xj!`Z%R2NX_t)J_477`@Rrdl9S5c#gzV+N)*S6oEg+ zeu^7yEBHqS|3Ek22`h z4De1CBSTB^Q7;b*+|>o-&Xk>PnikB)DoPIxCo_jF3?`4@Odc?ss4WsO+vqHD3xyw- zpsfpvFV_SUhpkHnehU#d140Y#S?qX%3DgN`d_YZ%n? zY+!GHO7XA#y?%C`Tm>$U)ao4x>XUQRAqDd7JJ=C&QUfc6P0f zzWF^n*=!P~ch_M^oA71~DMoOfP!dEaIXy{_@N4=h{cw(``r#-ZF6WD(jzFmp6kC|1 zn|)J)DiN@!UF-`nvMs^OZAs|r;$Z8XKWz5Tq|01fC*kZa-}SH*-4(Iu`Kemw)lX#cB_CSI~Nw!xhEdy!p(EdF0}k783Lx>oS=ku3_Z1YkM2wg+~Wh?;{$Tt>$#s- zbKSlV8WmLpIDD3w;=DbNn!M{#zXp?@>>Q&EjH(l$xyq)tTS9!B!k$JP8oBYDVA9R} zJ@Iht`HXh~GKLbgLxL^+{WOZAFeq_4rk!`FbxD?2jUTv9V>j2YuP%=@RcG1kE+?U> z22~rv&86g-aeG0uP(y?EPuv$e0lbl~%q>Fh#% z{Q4VK8lscQ%<~N?mHbClyMO3({nhukv^265vazyt`j&9|_b@Y3*+OZaADL%eRd5|V z(fML-pi44Q?5$k7pk5k_LcL5SfkisY4tY)A;^S zsI;0bhP#YH&+}n3(}h&%qT?1H94>kC?osDdY;&3=GZa?fY>-|!`$gz&RZ=AV>CmIv z-l+sC4;BeqT1k@B!jUMs<(`B2;y$Y`t<4jS%=hdmkmrowq8&WN2{R(mV7@uxxfkau zCNzZRVwTK-NidYtJSL$QS8jb&Z;{5xkUZQHm~BAKwC1sT+%XnV&n=oxxT2D&f=Z$q z(OSv6m56up=(It;vf>7SUh@2&&=KHrA3=deMRDtk%0tk!JPw!=eaG ztdc24>`yifs8?$(!5mokEz>dEwV}{n-6?2lO94`#k$)tYV_$iE=X1Kx8#fT!5Ywx^ z_&@96g_!pnT3{%(FWsE6`eJ)haD|Eu^}x84m1UGo2rA zWNjBd1RCaI3>ROgtd?6hERjXq7R?2CAT2)I^wBRuD&k_J6DVSIQgqcT62iivm{5ZD zGeetqgg1wmTcOJZ*$VcGSkQy)@4qC*`CjwkDCq##s2b`^r%P(rN9SCU8kF@&hz+om zOPg1-?4LDM9QMb0wZ%-mF-%TP&vG}nq+GgBojBX$9OqGUMt($A5eN|V;C_L#enNyk zi+AO^0luH4=CavBQb6l<3pCcU|L&(R9znp4bTbA|HNEW+s;dcA28?dLk01=MMV)xv?0vw+g~0@g0hMI^W$AcYK0&!K2!=< z6paT;e4lv`N0UQHgp^*o$Wj75AP|XtbqptTkoeRg@`6Cnk2oUcLa0*imtj$Bo6q0v zN#JkTAlkQF9mqe@^T_|_V$e#@l}<*_RmjHv-wYEq2v@8{gwJdmkHg0y*@S*Oe1RB# zY>lQkQUU@r;9*)>D=3l3LB!0&pDyc9jmW>%+i)$gSeMtYuGEJ}A%M!FUI|!RlwMxZdi$Vy;K| z)VA$mT()Fzd__AUVXy4U(b4-=?i-OFFSigN&v$oZZf{DEqAI|4%7cO%$HT=c8_{6O zh73o;>~K*OuJXOW*Dn!aMy}!_P`JjoV#wTOI}9iY*mwK&AD`HKyw}5MFJ9SU&QA^m zT_f^%&-vjf-V%e!w*h|DB`=g+L+;&EjN89TwjcJjAiE~_uWw#xLAFuw+ZwKf`)WA% z+&QfZKv18puTIyNw%1#YmY1O3+rV5ZH&|9U=v&J3P40Vos_x&p+S}+*!5=Q>s~3pu zcxu@ge`y)(U%UHa;EW2dEg?NQ^JFmqC+IO*kyZKwN_oUBZH;tE_cag=`6lS45{dRL z?Z+bg;V)3*v&|wB=R6CFQ&nTcWuo?Nq$yNQlF2V6GQ%hsT8@oC@EgKq(t9uxeL`Q# z=_0eHzcEyfwppf$AHFK!B&K;fCE{MuFQ6tWTNjniY2}Nr86Egmq5no22osu}H%WLy zPEheDZ})T8Fo{ZQhQ&XZYx7MQ7K}$=4WIj$7j281?6Iy|q*RFHIQvUbvO-Kyvesmx z*Mxfck_?-auZp#!s7E%t&6WKM&Ku082XXgdC@2Cd&t(_1kI2x zvHiM|IdL8$j1`AKPAUyBc9Loxq+$%H-Qyj!4**7$uH@hGgFKT`Ytj`58nq&s?=p?^ z;%S}p_I-1lt9y6ApXOFzbAVkKg83skb(0EPkFg#h<}TgZ)M08aRGLR4%4wcT5%uTM zQ{5pi<3g3x{2Bc%28<3((CrdM;1!yOle(s)j>nhMAry^O(W%)=o%1l5o4?hc+ahx- z()%?I_W@ZbbLl*+YTN6vN*+Q#uIx}o8M8~Z-t(DV zSrn0|C4IIFRZ*i-TC(3mrYiS#azlGus&K_U|H4RZ*O_o4mm7CFP|{g;L~~UEOGy{dC0>(1E*UaDQX7V>;Fg|@uf2oia-RyT`eg>nRj@*r2IP>Vzz?)KD4NM)H4;v>rpM(}V%d2+h;5x#C1Xmo0 z*Wd&BAv4k`?92+ugS06lskq7_E%p_MVk(q_iy(O`kilnE(NYL53U|)DTvRU)K+ooc zigJH5bAC1>22_Z=ZC<^8Kb?kyjJ`^DppMm<;Rjq?G84%4CrK0GR_pcCA&R2YG|SP7 zMJmlV_YeE#b|6a`zt~c)Bah`&;7iX=uDoWPGibE0mhOfnItlg*D+?otYxQAM6@rN& z!*SIz=(u@M&u^WZlPPSKtc|E;jmJT{EKAPj9_E6CtRnPR+-t^t&-hr4R>;*30EZQUY>vf*}&7?ykvrHchZ= zi~=r+=^;2$<`rG9IeSNGxN*c3BHAoETx7$@FrkRT=I>I>GBvkM6xao?+a^Xex#eVNf*XjV1j4}G&8ps zYV#?V7b}VqswQx6!A((6Zod(}Vy~hcnAjqz%`7Z(;$-!#7G|k@o@&JnFU%F}xhEu8 zrxsYKH-oB^Bd{#VvHX0J#|r-u3(=N`y-_UJuN1Ws*B^a=*yWqDqPT(wGOqG_v>V{e zOm3LoPT~*BX;y;d7Rj`7%Bd_Y&G)7;nz6CRFo>^!NwV9C^DC`G6KVw_7YuSd;8_obLW3%C(t&knu|-)wonUc}M-FF`R@0zyUAxh#qK+0S(M|9igx~CBRUeNQm}n5_r6Ki_`ZF$6%%|MM1h}iGu!muqJ%f zaq*J)_r>Lh?TPnGz3E6E1Gf33zA$IR-nS@yO0akW=VMghok zXH{mo*~DG%(lGryy}72CP^Cf2{CQd?SCm`s4SL#L!lV!kkBbn2{#i(vD4{;UY)muG zgC&^GdSprG#I$r9d6M1fnT2-9V z>YH~ZYJWvacxEgiH{yG#5St_nu~r|-e0p$Tlp^)kHY$3>o!MBzEoXkadB1TfYS&}O zVXKOIK2+=7pcXL}Vu^ppCOvi~#Owe?(7701oZ8^cvPB*W=b8dYOB(#bHoKfY$~1gN zx;8jxE=~#^=wOGtr5Y1lR;=Wdzc~Af`qsTPpGqu?{!@vTQ7BKvcuy3vRc4oaHwvp+ z$#O%0cvB+=RVFK!bU7wFqtjYZlbwlzE)N@ixgB2@AHMNHHn5(ZE8GD#-iXw>KBBVF zs#Tq%KN1Bp{w&z`ik)+Vx_7O~_NsnTq0T6?diA(y8SvBozMK@}f_oRJESKt7IaLQ+ks{g`d5bbSy4yOZ zn(}o}TN!O(J>ej4?Ql=@ty{vo6Q(+0SR}c5At?E8844_$PSd#W+A{1PYs-I-W&Xe2 zbN`n-dZeO^-1;}7EIlOvxS5#PvzdfH;BL0=AR|Ny7%_2PNXhPJ90|3(WkXZ>gNYq~ zqWccmO@7%FCun17jO8eojfwGg$7*Krb~i47qdXn{w<3(XYxZz3_Y=DPZY1PcCbv2C zuNpdTxw9@RKYmbXv8ZaY57s;7k;4mFbCyJ1a&2A|CSLgZu4<#sd6KlOybPtF=}1Sh z46R!oTQyfCTs0|dg%a(n68sBRI*f`gz9i`KsJnw?Q+JX_HoL<(lgV4LzD{J#RIj}T zX)5Sb;DIuxdZeH`A4Tb$etFG=ftp*JO+1(B#`19D&LxV&W;AklAG7(`R`X95R-Jeq z?6+9lQ3oUROX0Ixy&Nv~d(kR9avR2&-AKNs+GrpwDe@Kay_hUYon`3sa6~}f2xW1S zb2D;uNF4adFgx~0iwL#JmlqHIenKz<)6x}(HtQ1LusF#c@L$Dg*DK53r zBlHLKUv&=me|Hw#CCs!7zDHh)-<8h)Oj!K~z5IVgU);>W(Z=45SkcJpZ^pfuwTbli zSASuE|1|?s)l&P0VSLJhr^17R%a^KrGqQ;Vux?Z>Vo-C%8Ga)cEV*O#5kSk5RKwyt$lwQQIBCdVJR_a->62aQN#Nk2k5En715&BW>x>O{jH7|@*4I8;(ph@)RhPWv_49!<-y?_<)>p`iT_a0b2)EQe2-F{unD8X zHR7IoE=k8l_b1Qf8pL^UQY8>)oW|%Lr8yy;K$Ci<_Rp+@?=7j0TMZdcJN0IdeVx(y z?-j7{COqX1F-fDfqL~PO(#1SkcOiUEwbeLE-1S5%JvSxy^Q9so*`HmC4H+=w^^2WD z5tPlqZ_9=nlp5~mgi1X23q%L_Bu#cN$@!WS`i!T29e3RtFp)fP@YLIpLi*`TpH2?xn2pPbq~&VLpSB`8+7ktV)lS+4y3LM`*z(w& zo=nlwGDU4E)YQGJ)=98RNplqAm8N(q$0`pQ4X$JC7nXahV3FB2-n3uRT1cO)zR|c5 zreDwTOhcAXzWzYen84c}0^BBh$pCl}Xj#{+BokMj-d=}(kabj$ZH$mZ3}O2R`?td& z7xr6it|v0O1qIm!(Ob*ZYbqR1dD^7K^a`#NWuO5K3>j7^!UUU5+OG1OM*ETdWFcnA z7!R6MDuj&sVBSOPDH$2VX!Ih9_sSal_vno>Pcx0lLu8(gINlsneAsp0_GWUtqtX=S z7d?|*i1-n@RVtD&+-%k>PA%9lJuJeP2od!CMG>>SD3~^JZE$%8I&Nud z)Z9Y(S}lP0qs>i3mOa_{f!d}<1xUt1Q1nCnAMj3-{yzvjFQpV4hMUH5U(&iP%?+~GLhh0i)l>giyBciBx)&xyc-8dqc49eg0gs8+*5sHDLnsC4TR?Z zwfpk#bLn4Tu*e7*$Ub`bz@H!Cs=c_qfqPRs7{aq6cDoSzIOdXR;)sRch(53zdHpb` z<|%`%eVG1dcfPD2N$7UAQ2_)TE9K*6`OFLvOd2+ca7&wLg&m`=8tK|v)zYyF?CVIw zF?!ZI1Y_k20`gu2#gvQ!<_e>lAK>#OKNUe<21VQy;rmfkZm*y=huk zTvxU%9^Q>+WbL@76YrsrkARPi+A{`(SqzAAVx!~rhR4+AdxqC_E7Qk&_tyv7Z-NgQ ze@Ll*M(Z&pRS{Vs#%<5tOPR&{H0m$eY4Zsd~!!a!TvbG!_`U4b|JYAl41* zoxW=Edlj>;qP=rU-KCjOG#2auwgNX1Ou>#Y8`eE7SDC?P7RP-q#+NGQZKYc;7={o2 z!E;RaLLZ^QbjmHoA0Tf5R_Rx!OE=u}N;BLx?w=*`S`6C#F4`df7i;ero$0o%i&n+9 zZQHKcwo|cfR`|uXZ95hFi*4IhC6!8YGv`|Cw0m~Dd)~8eyFb#t*7A3Z{ywAk{=nAR zY-&P}Q*T^9qk(U`s5S77fHLX=eg2US*v-dWBABi^!*;fkW54o_PMXPSt@NydW)W%Z z2{QMYU|NzAtwx&utJ=nCO_RGakJyxc6_D#L2cElOhffXH2I4LPzy0>2Ks}j?Z zq&u~qlFPQNo@PJQ=-9 zyld~ld7(s^J2}Dio@Hqwj{Pg7x#=g#9Q0xVIaV}-?a&vr1y2DWH@HFyOIlaAP01Uy zH~Dob26HGh$1TCEh-#6;HVVWR;0qO>@|nK&JVgS?PTv5N*)`Hz8u>H4n9 z6YNUaY6^zR+=>wsj_cF`wOf&#aq{`v4JYuRTcq5~)WU2$)~+UNbe6M9K3=j;OXUHP zj#?E5|3AvOu}%cOaK}Yts>W;BIh^B}%=XYxfuympkDdh76x3TfLj&nXScmDuvYzpQ zzhy@ibLo@M1v{^mSk`Z=#c1(8D39-+)LmIu&3g7QxUjBalTBBJx954&#N8AWBGg&u zEa&fyf`WozP;U7N(BoMfhS9ahMmsE>`KZEuO=D3}Vq&l-Dm-+6u{SX>N>Nl?FCVJe zF@(nIK!ux_z#M+DGGmnK+rbM(yQ)F_an+P}Rp4jB8~(LX3b0dJFJj(%Ze~k4<-K5c zb0?&H|1%?Ezllv@KPs>$TP7SA7lFkF<&VM1Gr3>!-ZGD0(00XhcFguZA|Jteq$H5= zmJ~^nH3H#3l z8AsrNhMHjtx=Q90k3JMd-PE>2woZb^mw7F-6*udM>f03Smm>@b?>|ZrJ9nGa4xY8o z^1EWX*rvCS6RpE7oLp+V>?L`vIC{-wi)t@kX?jJ^x1&~^9El0u$Ft4eWEQf#8@A(5 zIf<1$6>u9v3;K*jT|b8aR17gUybM#*H3EL;uEgO&0*JS`)PV|lYBZ_)yWf2x_UF-! zmsoRM{TOqGpy+dtMugz2dv^!6sA;(q}(dr^wM9`!=AyyDoDpaC0SfrVLsq=+1>+YOE ztmO`IIrE~MysQ$9-?w3zaQ7J`u@+#|-WRN-7p!bfG_%)-w}im+-F{sgkHrsvM|9Y+ zt1~?YLAM5NqT>D)Uma-?8s1vWRL>N0O4M`ZpJ~tO>5h)>mw9e|&;u7@gs;0IdMvz0 z$W<3m1MheJ=p37S_5pQ?_F(Kj!#8QJmJ|D}Q^p_a)T^y6l;tr=m?y%kg%!^i;j&FM z7mYl#R3ZXpQe~nt{{|4TE7k^Ytec-y^mEWJ#u{c$7wXv`1F+nOguvgs3nd`>%PYK% zLGk$o$wP+v7Y2#}!i9#Ufq$oL2Gwf|{m9iT*WDKhYlmjKI*c!R`@??JU#<`itZ2Ws zCi}17cpgz~!NL^?GHXS(2ccbtlzk)U^HwZ$N2-4D7CTuM$#+k};*o}9R8=eV~Uz!_KFKSHqbsV^FC7K9IJ_x&m9$g-kQU_;Upr^qp*f_?S+wL<@frhta2 zOtuImpHq-aJ-kjIJl|oU_}6k{?*xf*0>PfjxU$NXy1z?^pjG*g@8@HuZC{`q}vW z8(LLmJ{37=?^jJOCtS%eVT^*JN!>1Gd#GT^5quGJM0C1oaF5!MrK~0dFhBZ43&RopSbjdw;?gBv64-J=kx`Qi|)<0)OM8 zyqcnI^N$qnBug0=#*1U7a?vn>8^ua-OeQ|sS=jJ=6EFOlNEPZWU!773j@P za&IyR776N^3uNn#--74#=5gLJQJ`Av40+!V2Dp z^{DRMOfaVN0@%V|)X2rMO1kCbJG=85x&oAs9hN!yA;hnfSbmAS4m^b5`hGZ&>gmo9 zo@JElm{wcpm=p?Uu|eq--yBW@F{pPy+LIROo_5 z+|ET=1pzFrpQL1%=I*UO(>(8V?ND%Ev z81(}ry#$U(B#<@D;gI6)Fh9*#K$wHua)B|!s`er6@{@$k>5upXCh3VhpyL#w6CYFv zyO1Zo1_ESLQrHIpJo-Xc&;Bhl`UrBSbJYuNh^V|bVBR-LBp!eu4|#XSxc>3pT-jvS z2Z^k;S>$I;LCS2h1{+O;)Ydp7q{S#Xs#`! zSDl1y@G8BG)b>^nP2lW6{_8lgoXGO8DT zczOJCyb00&TEB911J5NthZ3g$DDL@xsn`E#k(EsiP5#=${udc?wi5qe*j|4I+rFl?F~rZ;ojn(F+;^7?-J z8)64-T|m7~O}sG*BR@s5QgkDgLrMJGhAzrUiXwcpeAY}2V@e_SMYM~2j;DBi>%36va8Z2R!06f;b8vPNWn7B9z!S;xfepGDZd9?o+7G3#XL z%ePUd1UZd5m|TY^Lfqit*Z_O9SE`#DPx}=yWb!Ej4e+EF@pL>OyEso2I+wQukTLJL zv$TJ9dX`3^#u-d|oS}x}?&C&rz-&rw+8ihMFOGXv-u@TH&w0_|=ih(dlF zAA`Id^~zj2LlF-jioQb_n;=xr53T05@K6*s@&^O-ZQ!nyFILd~S6m&qkV6vHk;l@?z_96pu&*iUV{@B!i+G0C6Z*$OJHxe?%-DOjUiuLmKiJYEfWr>U9g$ST@Y z;W?H{2|9w-taAv|b%^*Q0of^S9q;eok5=8ip6UocUx)b0WB0Jx*%#)_=MCt0>+Loo zdk)^9jsVVLi??dsqZtFI))H=hcT?(G$l-`2v0xfUTvdqrK{O(KTaQfv$^lO?>=Nk| z1Kk}*ixWs)u4x|z|JJi#V!0ZMeRB6JJ_iT?{pDo;?=Pp5sk5t%i!-D0-@kH(F2)w0 z=AmS4|AO~qRh_@cQtvGb007a5+D}#0g*j;zuv#J*a5Qj3x}Y+|_97Wv3OTC;mKznH z+eJ)+tXY<`sQt!oW4C0{pT0v)=~stYjoU9%roWc2?Z17A`c51%kCieTm;?u2!C7Rg zteeV2LmH?`I^6jJ;4I2?Ky4-4NI6}={}gFNv5x3$V7ch(aNfh&!e;tJRzh&zTP>x= ziy1FOiv%c4QStJu@)jI{|n+kCb%{w{jt z-DG*=V?TTXhvNt0E*E)O`l?u{mE>PO?9zbq04&s}GdJkhllBe=W@vfIcgd+nHblD& z<-G#TsvD}qxi@JIUw5U8_;^%5v_7S_q3uRWZQ!(&hQfXPvNRS$*xh<2sAMXE65{60;2Rwk@{#6GEL?!YUa(;)lcg?KlD)|QQKKinuxalX7Q_zO(3a@HjJm=63Io$@hd71H8umg~M{j((fzRFyXJcPRuyUbw` z<9>(NW21T37U%ThW2@Bw2`>JUIe;O#PpChr4LxwF74xr+$J=~ETpb134b77HG? zRYAx+$C?Ec{L&o(u}zcmCuSs4PtrCTU9|FM7UrPV0%rF`z;w0MPfpSv;W_N`C2SXj zL7AmwWWBf-*hS=TLGpqs4L;fIvWy@o_A9RdYtVEq)N!b_C!*~eBJr*{cepjzv5Ju@ z==JjR-$@pY}WEZGC$?1~Pt8 z@mLLKC^FR*6&J@qWub^qbcD5nYh@-k)CL60qPcf^4&zc7wMBZPn2~Ro$F;UGU$wQ^ zsbhO-aJ*n>z}u-WCNPe(x=fswTure)l*$v9E@P8`SCU}0Yq+R%S}l$D>+hS2csQgC zN4wgcZwVlYZ)99C`3q5Jl+?jOxZ?>Wl%tQ=RAkDrltf$t>K@yf^z6!795MjsW%hqO zlRLEKR;HfW923vpr!0wowRcjpapt}Y9=T{ND0pFHC0wKLTD0h6SYr%WjI|o9nP=tYCr5GN4b`c?eIuj5KtP89Z=@oftMr=L81q|3?{#3k^2#fMkq!5b9Ma9 zeRl$a#=q9Jwnr)Pl&%4jdk1;qpPGgnz7cnAi!`-5r((Uf9Ow-`69M+PFHI{2{K28zz2R}MBT zw35RyEi_2kQlR-o80=K4QzvN`cR%kTK69R}l1Yj#@D-VtbhJ)$^Jl;OetVPna{OIl zHe8mLCyt#7LkyKA06Vs?e-#UL3i2p0AW})UKR93Ro<3)aAqqNzKkcx3i`EN&n2idAlD7O(xm} zk4I4>TE1PXdb#YRgE+)uwQ?t7nj>lwl!S+VSYS4onR9pPGq>3xJ-QVNn!w55)I}5GQk_HAGEjTRf6 zEF5RidM`dIY}+_DPf;R>#8VepEjN;D-#+YU=YI6bL4!;3M#AX#sDyL*ddU#l0Do_g z(_!-ny_Hb!x=!MI7xv=7bc^ZNx!Jk*#k2YWvwgxzyANLyPumK&EfM>Z z^mpJmkpreJvSG|2`+38FzAVRHzK1HG73qya21Oh6Zkzg(18r;hh&f`q)K=Vra;x= zfI@V~)4v;**u2fykI&)~{QnuI{=HE8d=vhwcd6cV(<(~J+&}{_LOWINqs0sX(FTV1@T?Q@*T`YF}@mHVZ^ z0~x3$J7ysw9!NtPB@3rwG!?OH2RCn&W7)^Qk($2~5FE-f#FucR*-t0pW56om(qZKZ zM>ma>%}O_*^F6HSZhxKa@B+l#LWh;^y+M0xcPgdDID!19*?Gp#YHq+YLKJeXN!#Z$ z@GfO2JFEOB1Eq}Hq+7(9esZ0fpW<$e(osrjdd0wI#p%y8>mT~dd|iY1?qo6x2O#dQ zz?EGcj)CuE5>%F^*}{CTtJ{?x0NY8J9QqJcP?7DSDY$CWZ;>74mrnO_hCGkooN!}= zwp9(Y4=>OHZ1wd;D!BTB*gBV<)o(e!tV_~7eDq8_miyLSNMhCC-DB0mI zgJhjt%-ewgU@cJaMj7s~kYiJ1f|#`7EF__}QY!%T+l5ONXdNgf2dHP8m8z9@)dQT^ zpG5V#R&Sk41QlTjk!CRMfqWfyMklh_?qspnn0iQ7p*-dEeuDPGN4bLxKlP*wVf?Z^ zWoIMj#St-N$V7W)XoNq=cuw+9iWcF7A3zUWJ~ZkckcD$kX@ePX3J*Y|3Npoq3_h)7 zas@@cL4XARdKiNa-VG;&>fskA^DG8f{D#$pS7s4E$JyH^3cWc3(|!^ec7UZO?CwB7 z?u}d|s!;)INu~n>1Ixfe&|$i@Dxou{(Wd7Xd#$|daiLbv-f0xd;NS6IC-A_7Yt zolh~oR|Gs(mCB=PCp@BP!Wrfc?z<#9y$qK%g}dG^%fk`;_zetP{e*4^aktqu6kwcu z0Ok76?-mD+mli1n2$`0tNf>vybWVcdt_&d@?NPUiwHyWEjOPYMyg~-L*vt%4-R%cm z+fT3ZpQX#e`$t#{sYct#Jii7J2A%=-o}%SgH9`%HbmLEL<|`Odu_;f_<~8XX6;`Eq z%97Z2tIMvt?H%2taZ?lE6jq%z^Hn)ZX;E>b4NA>z$Gj%bu3&UOwqOLfAPx^0bfs$q z``hI%95>^$?N;k5N;E$$%&;md)J)T+*=)bB zl6tXYp4KnJe`#1TmxK%Xkut{lb=&?Lnx8GhjuWRh2pW?$n_o;@?QB;uCk|@F;ynDV z{fCbg<`jFEmh%NDG=dMI3kRmyb8Inw3I4g_f`Xpv9!O(_>AJg1z1_$pj#XDjq|+W6 zgE3Kvlaml{;#%HsTZ;V-0h_G~A8L~j*n0nCH(!${G}#83lR`CK?q z(d>hBXo*-rFvp?;6Mu4-3tle-VgsZgcq*ol%09BsJf9&u#fAg%_sl=xD@%jOd!dx_ z!fVwOm{Phx_=x-&wnNr7Pj#w#F_|-4Ol@1_4Dwy${Pm=5p%UVUGgZlWVP9pcC9VDJfHeRU{R9#I=^4kL)q*vKPGMAZ$sELiYzM^0$%K)%(G$jOz+@- z6^k5+p8dpUv8?~&6!U-HB(eTyA|YdsDuwjE#{B7inoz~O3%@0|@(9WjaPt}F7Dilk zztRkj0$DWGX6(+EQ*XA0`OSE)YF-seZ+iwUtWrJ`e52|eA2tLQ&=#^`1Gc}};>oB9=XmJQ7 zz87vowaw2?fx{ZE@SADUDYyhhch*VMo99+IZ#p+^Jf|N*YVU`#3PJ zqP&FCg_9^;!%bw_z_kdOO5Lu-Aj2JQHe(wePwOvZ`wM|^DHVk5kIv&Zzp~Dwf(7~> z!(9d-|Hd)akOH38pDE^0NWA+pO^=aVC|J1T)=22~5<;8lWh*)Een~@(V5e|Ypz$BW z{IU8lM?iQA)V(#o#(VKxb?1hI37NjY859kPy%K3*Utjv3(L0d>+7uVK;T{#ltek3^yb`mPU`}4ZpNF z)?7)$cFEqkF1((hoGGA3b+sIw)p`Gc2R%^VKUL;Mdt-o8OwH74P!6Z=Qycyv)Eexi z?7}l=y>JMK8C;oCoOel^>=M4{G)gO2b_mW<;o!0QooFWY+uSjjpagHShDhieksi$t zFj`;XJ^pGGF6n)up)crBvm=F`I6#NFy!4vzPkT%vVX99Vn zT0B=F@;mc4@kA5z0_)q?H>KibWlKLcCGPA+iH!m zXq`NX*tab-O-~*@k(R&)OjhqJm)uTx`tL-R54)QyR6QZ|ZUn1_0Ykc;>+g##RxLWs z5$wEBcdWMEe_j}VW#eDLbeM_l1U&Y-pLA;N%Z<3qdiP^PjK_>$hw^4_7$kZD;u~5& zWm&e{@yD8J7~UNZSXU7fOiPvuZuQfnMUH*CWUn?uw8@6Wfa3bv)v^7gifW_J68VgN(~O!H&vqZ)v5t97Xe0Ru!*>-7|RQ%N!ItvASF4Ik9fC zA9Mr*%SlK(BZ!tN*i35!7U0B`S2F~kH?|D9J~epL?TwVrw%e`0)P?JDBO+MhQY-gD z-Z2=ttt?O{(-OCEXdsquW_qc0zo+r)f3pm0r|I|=hafh64B@27c|+fw*x?Cr|D)F! zhs=1+c(@mg|8GlRrC0uHZU%IOvHx6}1Bibl-P4{83 zLbsg4am-u{VS&*}rIV7C{zSmkZSgq)SI;b%+))PVGP4ECb$Cimv|=%iYVGq3{VXlH z$8hcrp0E7ysmkCCX@?9iycSOHyo-QHsp%Y(m)S(>cgER9g;%r^i%YdT>Q049oX=cY*I z))NU#O8G8P%G26R*N61RAN?M)6CVB~fqe4^iR7{BY+cxL8mclzjK;*nodl?x_U~W! zzr58l_1<^~hFRN+J^&q}P`Xf1>t`<1!4k;s)FoO;KFWF(B*i)D6n1z*3q~+_CG%kI%&dB-D?Y zZy6JxUkjN3uk1u8SG!MUv@@faow1F*v#H7df^&^|MWlDTOqA$)B4sq6e8EeZ!sixE zF;OwtEK&mgGa6e)4`#Bg%mqm6A0aZFcCLj55;4{vL9icTJe>JNMX(nn?kx9{oZA!G zN5E^w$}jT9!~Ov(5N`WT0c_$fS<#PY;APMgiIFu zXx~q&U{V!0F|BpCTCEX0+bi`q-3V5c%YLz!V(_60h`i^5{5eEPHHz;DxUAC1bU2SW zz86m?ew39mT0vDA&08iRhXb}!r{}vj)@aDk+w$wcMq$tdb5iIpy(km$EiXGJ8y;7s zxnJf{Qx~J)W{X9r#QlC(IqeEt8cRFI{k4{8?H#G|)AmZ1 zu5~h_?3<~bGCoCe&K-`CJQAS@OKK_l$+l#i&(!W?ua%)>)s0RERz07Yq*5aL!#Q0);80lZ>+Eu6E|FPmuXaf1Y2m9ysbQs(W7 zG#K9keZ}`s69`8VMLGH5`#~kByuWyl;)Knd2vL~G6k@)HDQUgVN}TvLOtLF5BE-4I z*+fr?aN#X|6Rv8qGNA_V39Ap9>M+RXr>XK2uFy`Ssa8)oZjN$0p@)#cwmopFV0=Wz z&&}@%(7KSR{w@K6XkdM@nVUPb2jj-#6I36l2kjI3;-}~LLy}(-Jha>c!xIk(wTh_( zw_dLJhl$-eDnl~<0>v;?Qj?N66uMTkAt|vKLXyM@O-%&5&?7+m%m7zv(9xfZMAZ>r zjYM_f3A#ZZkn;HV?9`9+y_E5@5-tA`xBr1B{9m0B{teh_{|Y1yMZo8QQ@kQ^7qx_x z7Q5*eJ0PJT3&xVR>MhOBZW1ZUHsdKg$-M#ZHX>P)GaD=FIO2a1CjeB5n&2kMqb)xf>ft5}C=~tl?gyp6@E}X57+1ARF6C|Lq zl2Fs+bfrP1J$V5?@!GfWz)a34+-_B#!tVrGZG(#jD<+-wuEuW2;4H>Y3(#zL43lHc z6$uv|Js{^%B4g-QIe@m$ zT@ifPQR`WfOtElYE%Fvk(E8v*!4RRN0q{`-Fw5#D=AP-|2ezxwonO3kKwrZgN1xOP z(_4;%sdB_9i}$f#`X~*8eP_1eDEi`5W+;}jSJ2~-IhL}{V zv-qH$^e;}>Uxv+9tM-3J7)F(SrjbQ_wxO;+AIw-}_0?PCSkiemHL`OX;eqv|9rp^4 zZjCW-Ek3fK$nV2tV(lK~Rq;=B@_epp5)3|=g;G&-D$Bx*CZK9@FjAEWpV1CM4;M=5 z78fO(1fF0PcJtETwLhqFngUe71Yj&;{iWF4=2k+GgtSN3WK!Z;;96%O!_W~21|uM? zh_0FJUJZxAVDqws))GO;*>e1QdicAEkYvIf*xx{v;Xj0^I!U`fD9U%2^e-U;ffmc% zq#h8SwZSFRP?nUKr%f=cg%3SU;=Nk&5!wKh!#F4wx`h2Eg2{xu>?|6g)g^)aT`cJf z*o<$25sqvAd1_$yamIdWOjmKOviDrza65J(!Aw`PtWQOu4 z=5UnY*j7L0GgBb*6^9uK7lFZs#K9y7GPb6Y+x^@K>Ee~@R`XR;ya9dAA6)Igng=sJ z>1tbDO@EmjxcV%?OGy9JvSY)YfbdYyq8mxtSh?)CI)QZbH zBBZZDvq?E{I&h~}m%+sIPu_usU8MZ8(GaD=z6YfYmuS4*1DmKdMN5>K={JpJ!<8|$ zzn`>3iFBd>!lMBH!u<-_7LsD=+x6@qNE{>hel;M~JjEFSYhaIa{{ zgYh@z7y(SHmL>7QrY(`rwqpN*^xQ5+J z@L2fLNPN) zw@thOi?xxdeASk(PvyYxZ|6-rIkN+b&oRsGKN3Iw&o1M?Z=vUBZiS`E|Bc)Esb53Y zKzpZXl>&SvL1NQp*BkAfI|F!0B@VaMh zy9`n;qmlc%v8~c!?u3g{#ef;fdVH~(YX8IhI{W2y`&;f8E`R~p#Q-o~qSL<3BB2$HV``lLV9wk;;s zV}#F%?~V_KFU|jxjv`uTkx9ds;<2$)XHCP;%* z{c=rZX|XiK1q#>3!Hk}4Vx?z`E*&SjGd0~*EUhFO~Bm{`9j(^i(zxY%0r zSgI9-Ud=D;am6N*7kvAtJ36TKdZSmwn8QUYXDIA0n*%NBddmyj^25RwNaWf;hPDo5 zHcPC#4E*#^ngTGnLBBzd6Oxo$W<-%-XcH7y?eQDkHf;dLNQK`mEjC^04U} z?$CGE&CG8OP&^n2>JQx%kd^OTxe1Z7QiEM!je# zB8H}5^DVetyIA!svLW59GKICIc7|y-OGIHZ{2VYk^nOfK7>wiOEuctjbE}G*VmKt5 zQSw$Pa!+b{Le6Vq&4`jG3HlgZQxRM!tlCzETsjuW8gZeXXSq7JxgK@nX2e&Rl%keW z(qS>xw6(u$hCL}oQ+mp@F9Z zvLFL>L4E;9B{Px4bMWyoqQ6nJ9{16*V*(^r>bZpmMIFA@?vow1!?G5+CwSytRHUzW z7L;igMJrWBELkR~l@XTy$hbzoa!4(BK3QE}-1Ux{ zs&co$-<=S-kVF&^)_59$Xy=^W?>mHiOyWI(hG`Aj0FKmWZwU@b;Tz?GMmf8=jSAZvdgP4V5xiBN3O^JV8{;$Rw45mxm=#vH!5BpD42$uh&SH*vQP84fsIpeBf zyw@~0k+oiG);D6&$zG6Y?~Q0dg=2ynH&AhrtWflQg;;ZvxGGw-ZP)2D%QnmnzVXv+_qim&&(V!Hr8s0~qnT33WArX(4 zHg_eN>(B^!NO*wK{1~|c<_VuAdYdV4I%laGKQ~496{E%PCM)>C7*-F76aR}?D?kSI zL8+>>aQa+0Dgk?Agr9Cb+q<3#1RLfs+cJ_qY)lCyP|e&QVex120L3H$$I(2wp_@~< zdmh=q$W{(c(P*cUMI*SLBip>B*#z{W+XXXwtp!ARMPrs+cjr`;c=(!re(ls`MM{bW zyu@sx7NnR`)-M)qeW$0CPPRY@=L5R}!5piDw=c#WlSYvRnCA9J~fztgh;XDzqO=*5zQCXxxK9yTXkC z8)`ZIp#~_6HudN!m&H8R6pF#sV(2A(($lBM@72ePs{A%+F4>gY4wSOUa`P+{_VYIC z&6hdd-2&2^x(4Zug|&OerQV%ET>EByV-;*sFJyG7-nbS z!;sI4!xGOP%}W&wLBR?pJ`v6+0*SZW0x7rP-9gq+GuqG@dq?mx`%sCx>H{P`gN>l4~9cI`89Dj(NXsGy{0|JD=itl zBUZM=!>}15o6L0;aBAP~Cs%2P6K50EmTP|VwzPk3-XPt?`Y11`)O^6Ow^a)1LB^#i zJsFHL6&R>Z<+3VZ=Fz={yU@SgMasI1jHk1{b$_lZz*H3&HhwgdCaUFH#s2-XcPISr zbg_w!_D`s!qx>8yUi4ya?yfdNG?i6p6psW&Ctzi*AlZX3gQwc;mp{!sI0lEWCE*&&NRp8 z72R94arFMkG^jZSnn9d{L%o~ikO>O;s)VqAbLRu6?>Q|968iUVMxQu~ZA=B54mf)w ztgN3m`t5F)i~=!(t(1Q=Tb$@#U_xWAR0awt6&!uMbFZ)Y z{CEsQWLPsIpBKa*A!mc4v$_wWnDvZ*a6t9-ZpQXp7y=PGjv22o1&~L8LEpL1R^ubh z#*>OeBXv9M8W0-~_vS(k@_{(|+hTgqo(QSu3A;$lMK>yzq6XemIosVYWJ>gS%I{kI zT8>R`2tRR`m$C4^RD<7UciCWXx$~G9$j^(3tR;A_w!w~VAi8Y2p~v79JPpkeLNs0w zULIDba$?5)fYKG2jU0bP0fyYPgB&p=xPD3vA1 z6l*vkj!C)WYf{WkTAJYoNCT9|v9_fveVm13S?1J<)>TAEZP6Tb4pUo>wwT?`hUdH# z=vmPvmvG<6Xki0%r(;24r;?72gmtuWeg_oX5WS1!twl$v8MmwCw=Xv#A%eSgpN??v zxQ!-+8&qk%202|?AIhblxY#q=^I(=Dw18#!F}X;@D=ihLn!{*rYnJQTUrp&@ZC+w2 zNFY9_?ViGXqj`y8Mw@UGvgvGNa&`Si606bnRyjS?2J-sBtcHpt=`g4xUj(Ks>D)c2 z<iuNXioMpY0D&oE>Yz@B8is_@s*(Fk{p$$V;f zzI}Gh|GiW~?EmQK{`d3x9O(S#1f7NK{(XAIf6?Ur$|}=ozjPZfCPEhj7uwj;M7}0Z zTN{*|dsYtWYqj-OM$mH0?817jeF&jQcn1BWG|at@K8CVajp%&jV{X=c%9;7{zJJd7 zMW>8bQ=`v8K@0(xhHXP5uQE-TI;eq<7NttMwxn0FZYXXGx#h+k`{;2g{$-X8#6VMA z#>20xX*$k|hkuG`9);rvE}nba>(qs2Fq*dfa?_KX#)vmq$sdtj=7Xn;8uW*@<+@O~ zwH#aLR?GAp;A#D=(?+)=8d7E7H~2Q%n4g!KDYNXpTkg7j{G?;{ImG9v=;v@}o^%5? zo%VpC$Vqs}9-NJ^p{uCt6w6=HbpzRVcD|`JChGXv>@Yxc!7g-;m1bWBz+6-9CTsWbM~p!qEa3Nb zq^ya=Rowf}X+6LxR~;L*^L#zs)?9mKMo1#0&HW+-M4E14lJ+dMX%XxgG|M4!gG-j+8(xXznhRD7 zx^m1o2I&~s6~GN1)&*l^+z|CX?)Bj-0m&=sUza=#JXgf&^MZ%`BeD7)g7}R8*F=)> zKVu3THC2lvJu_Nm>rKYzwz=DXE&cb^_6+t1>ZQ9oC>av-{>8)Njy{m!5Rnrse`?(k&wMFL{@<2+Z`VY@2$l@NQHQTwc*wOZ+x3U2*1%%3BHdc;s}yp4c6(a(5ARZH zhgRLx=6?N1LGV|xft0E4%a;_LGVN}?N*SOXhIGD1-BDXN{c=^sw3JeBm2{>|zII;1 zX>S7kbfanh_+dPG^_v;9_m<7}U6%NGvrp+&vu_xqfQ+D{P1t-dj!jTySWr;?UK{c#oMwj<2tkr)A+;73faUax44&_J=++-Gk*b zqWjYYjeCcaW81~4Urd$1e&d^(Rv}x;=+oI~_*8%H7Q&eso`5w2k}ZdB=!2P8v&?i( zY@*;RU$h%Dmw-}YX#f`i=uKKfB5B=>ajr8~mm=zB_4xSK+9tZB>t4uD2D4mCl|Sil zL6kW?IMIyjpY{pT;4y*5Z0=yUupwWIo}0j!6aI6DJ+Y#Y(<~Hq&ojx z6@kMec7nP*9Ht#?xtEyvfWni5o8=Cc3qD$qRg{OkQCT2o3Uj|Ds9vyYi%-}Bq8sVL zYu*biv$$z2Z zkkHvQXV89!!bLmEkzOR1tXbTxgFg#-0d-L*$xc z4Dpri9>AJV$vq2T!TTrn#ScXiRulv-nIy#nJjniU?>oEVj*OPiC5AfGKcV11C@=mW zJ`n%zjx#LO&;it=Zq;Ji;oi$aF8n0=EWUgLfT z^s^Jwg6{uX7%FAX)AQnP_WNI%O^Wx1^P%*FE!b!h!lom6ZZ3Z|qy{A&$XzCMr2jA0 z-ZH4NZ0Q!o3wMXYp>TJ1cXxMphe9{*?(R~!L*edFIEA~r!=wAi_xjy)`raGye(W8= zjure^bLN~Qb7YQDc!U+Gs5YqCGLBqV55!@oji_>-J}&A`gJzJFY>$bWv%KjJsLFZi z+hdK9^>~7`)Q|;e9bINEcc%Lz$+}=Bc3u8P`gm6?EC1s!->%CPffwW1Ud{v4Q!b$<~g(FL1 z1maTPKbJxQgLi@E6ObVN{{Ton$Grc_Cz1T~m;VhQN$%CW(hcwzEs)ref`iy-I+Nfq zUbeY6Jp4I}kuJXdseF%hV6A4Jf@tTH%5=cRbR+X~dHexi?JyRJ{((pwnzveX zcE|}-19NWwg-BR^e!KjMNPgrHR}iCF!5)iH_c?zelBcWv=T1Gi>H)*%5Vx_0cG0CR zSC5h=A+$P~nMN5tdx|ntwf%7FGBpF{rjZaL*#Ua&8Foh#oVIr{|9BFi;|86Xe89o& zthn}^Tkv{PIZIZz&CQ}9CBt`#9htOVdMAO$!CUw+Wg$GRS!&6f&^%1KE)2+uKAor6 z)P|jznq=lrt_Ds@ec=*z3_6>s<8|ta%DqSa#V0QAbogwJ@1b>!9_tIbs7WnO( z0;f>wShv7eFk66e1d>L=LhSfdvJ$n3b{V?@IF@xJqRCWPUChS@+$ZT<6jiopTwN@lKN(;^k+W3qH0bPj9xZwHR z7x&Q9KT$VLtXTd39+3Q>@l^lICjS>{s`W{lq7Ho|j8#h%M3sqZS#>fGk_1$sDoBSW z>I*TyiXNS{>i%5p!oH-*B?T#ckrD)U}y%qGVFyuy-gb)7FdeEXB zA&^@LnPmZ=wJe*U=#ccOZ0M;;4p3`6hx+Kv5|^4f8kJ`|aneUi>&PN}*dxDgN%p@W z$rc{{i0YTu?x#(=)u23;!+Y}bdl?a-1A|9t?$1wBVy(*Iz zIE6Z)5A^()UxwFyAY|*fd;T%4%!ZWQB?dq&3P4;bjrYYA9%XULGaQpj@Ujf2`SgM8 z%aD@pVsyiYD^}%4YR0;!?J;zJ`u?NrX;=G>_1Jlj_FSHH#?$}2pRJM43bk(^+ze-o zE$#Qh*4DltgWlF|$#{yAylQ;rnsfm`+Uf!3tni3 z$s;> zYUHldm2lzMs0;bL+kjVsDQ!+IDth#qj!G)oxUpR!lYwb{gP_UviW2IQd}1dV zM{aJDLs3han)iKIZ;&jL*nH_Rg^dgwKl?_VGq{kop+uD! zm}&>c`%P@=o?j-kE9G|J6H#MZky8cd#Gycs@)Ir4*0$M#VahSdxrVHEiWgAMS$|H5 ziAh%*KF=RRxT*kgAnj*ucy9OQw=3?4`5o5_<@DfD!Q4w}H7D_OUO!z?BA^Gl1$MMp zb0-<^LTp!u_0{^uMD#hzR8xs@NJ3-cFJ5UXH&ad)RHUU)5_d3?G+j0xJa0E_kMGQL zm8H&cW~tGarOr!A`Maoc8ik1$_t^iM{()`O&tc@mn?_tpb7h@LVf5MZGn&@gi^kjv zpTvy#>$R3RH2+&vXaL@b?!ax1%USit&Z#E6$M3LCpL`8%ni4gso873$cswzQbndNh z$LeHVffqKGCx#emYP2YF_mX&HMbOM+ivAolw)=3(_CF?uP|g-tH|m^LvbZP^@zBdLZ% zFZ@u)pK)oEoR#)TP)&Ag9T}(ZSkG|M8Y3#;mJjt5T=9_>Ngb>8#{eqs$4@ zkY1Sm)N=4FzkInA3Xa|xeutn1*|_d!+=Ecv0)=25u}5+pz9a^{7{Y?!G7KfR!S}V;RC6B*Nr8LCro()M^Bog*Y>;A&jh{QYgaXCy!vj?tz2th$w#(gp za~ikCx9jd^d2B%VPJaol%W^5tCI=ds?O?Mih9O%qP*daM#p zY?;b^wVUfPqBj5<<$1!GJ4^1cI|h}jyveB5w5>)(QnWjApwim4C!SOabzj9Qb1bmc z-LNb(Rj(0)hL5qTu?{`#dJrk{dV{>Zi!TdjdNUR8WZZd0vM$5i4Tjxrh;~0Zdh}Vf zT>b19DqirMdyik5`&~e7wwhrjOiAMMvUwThOkSSqHkdTF45Z1?XDZw> z9uPe9I+B}8qChV1z=MqF#FbQXA(7;GQZer_m6o06g`#dFdc2nQ$4QE$p^35z(r30z zadgbWzbJRqsca zqxYTU<#=$cd2r+_Jg_@|f7@Ctf69g2j?;s|&WP!nsFcds&-ThoxytxL38$)cLZr^22;G)$=w(o0*c! zgYt0aR_dBk%*|19z#6k#XVGEWI&Wl~;}M(1Q630>8}5Vi2|3y^NHtn!bFdX&Rr}$j zt!Pc%gR0{Xziq*4ouH~j3i9fM@Q@9TJcCEU-$Pp|dAz8+q1`*?kc~$_u0lbr$?|}k zFCVLz&FIMV@zH(*OxLP=no#=I_3Ad=O%*!h3t^#!5mqMeWyabURf+$-aKFdk{Z$J)w z;!y}l51Xki?O2g}EzR~eB zpxtO;5w;5Bw{v|Fr}Xi>=9CxPq2JEXl$W(6U8u2IXH6_sxsnK78hw4hhKD#v(531L11CbV5_6qf=@l zWz4TeXm=;~hNoW%Ix|FM3$)JHemUlkA*w_*^s=_OjnJ?q?Y~$d@vlflMGryf&}s^| z`O=YvtQb3BIz1SMo7J#qh|OmUq*Pddn)$|&QNwZ5s~7IWaU!`v7{R(HsXbwrEjl2- ztb@D57<5du_=bRA*+u64WKwU)kYxF%WkrH zQ1e=-C^mz?sB|Gx&qoFl@ROVgjzen~*8YBnQ5)d$-Ygoz zw*4y4#C#qRWmU+f#4PbTEP0QA5lO3*w8&G}DM*uUm|bcNjarpueEhIkS6gn&Kp+-0 zqYDFMiwLnhLrQ7X_8lW8k2D)QB=o01&n?=}D|5MsVj4?v2 z)vH~GC@mT_-dB)J;nOr@ zp^-jx(sB{eI=PJGVnQ*`WTwXZue#H5>VM**<>6OqXqS z*2%!2%i~ybKM)v&0Q1P$fJ){dX9AsGiq_#Z2{%@63^ZhL{BAMA1$a1QwbRq_iAkQW zE@(=Rh_Yd+BcsmH={99eG_|0? znXANU>4^jqXQraP7Vm5cE~NU*Bwy2gTq^d=ryony$)gOBQKslI4auw%!A|kN3`|dn zG?H+0qzkFwsTRQr+n9>7#&A?V%~d9kBs)Se3G;#4gL?R!DpiX~Fg)X=lNUOw?WUhj z27tNW;e4^~XM(1d4EG5)Co5$2*Yx)?<1pV6+@`f`!dgWmyFp`k^CxsVSn~oWR_1=3 z0ClctiNMc@JpS1bOYD}mi1qn?BR@a?o!+DUU*7M(1QP{Kl&mc5?M;k7qrt@9*}~5D zAE{iD;+U-Ar=Uy}0ocZ4x`E3#TTJN=>=f*Rd@|Wk7=tCY6l6`h^+Nn(yH-Yrtv_hk zZN{+5$hnB~a;1`IWtE53moX|o9%U(YI^vipBG!mU!kW`z$~4zuPLtb>F(%+EG5tJ) zh8x9~XlYP1dJRGSdwGk!7_fUQ8Kajg8+|s_m4si6QF56ueQ^gBoTG)aN*AvTd9dt{ z>qx_F(lKDk>5_9Qn64vDCTu2!v&>K_gsqtOAO0SE^0K+SMe}3_IAn#LR6BBIE+wJ` z7I6N8M1ebN>mM|7xoeC)WX^emD_VL~XKSNg6tTZhNR2cLqhi`nTcMba!NWVkBJpFR zYzrj=N!el;eWB;OR%7{Hs8LH4A9hZ1JF#b#YjmxA0b0K)*ZEQfS3s@s1`BqVcQ8yJ zx1Nv`M1bqP(U-^a`q3YvIes5tahU2xPbqd1HRZ&4jjF?Rq%e60-%zR>TN(BCAt%>g zC~T{fw%f3bX}tV7adw769^>-4m~MYt9_9VZg1Y{zgPBgi*4ED1;E!c=`gdD1iHwP} zxt;O9|NdvLo}}}K#vNe8MhcVx8YCbTpMdBszG3^zbnjtq@m@Oh$sI=h5s z+Lhc9+_?Le`&o=5cSaEF0uk_6ui7KqNsr$=2dCNIA6I)+e#ojSHFre|PDu?iVZUWS`NS<0O-2vLv)u69n#+pUvL|l2II0p-{cGnk?fzz^*#3 zRzI}Kn*c`IE(Mj!j;mKElkIu%CuoS6wnTIhiQrLPrXQr>m6-V8PL=#9n7}+jBA@RYeX@A-d=}TaTh@bT|{(8 zCHvRGc86SoZ5drCc@;Kol2pJt1I*_{3}7A>f|gvbYYgfaYj1e(sONJp#w`=Db?;1H zh?K}B%I82JBy0iMn8zjMH2Snf?_hrvvqErWwdy`2Bld53oWD+Q|EK8qcQE{g-%V1{ zQd$+o`-xM$xB(=Z-DCnkxZP9ytCX<;_r zFy7#EEaWTwU9e>ymLp==RQ-; zxrV@@NF1$(R`Dio1Bwwa2;lpzfH!99c|F zWXlLUv~2=mF;*3u9#*-R`m}l2eJlCeO=e7ag1tuTG===3BY$nB8fai6AH%gq@Djcz z3is(>PlyZ9KE1}=0sS&UKhf8ry$2>*t9yYuyf3S=@xDxd5+H2FmhRK$xr&8O8xx?J zyR5`)xb%8w1sLI#(_*z_4obN>`mXX_FCYYCq^v*bm@k>})b zh%PgZUqk*TYSN>|*O5gY#Ri%(_4!{HrNyk=zA`i-8l)4>mXr+_&DkY#4iv)Qms}DL;0m#+%hf!5{#H z8sV8`0;awOx=!>9zKI_D=Xj*?n#;e!o)9{Ifz?l3JSkQ(P(R`;P?~e%noLS#9Zf9d z=6mKhbfa=AKtEA-HR3B6 zDob48V0;92YLjtIu{^_Cy2(^$8R9p^u*fT4(8rbW;V;a7IY);n*+Bn_QMh)!fvMq} z@CZi?`;>$(3bhssMo8I6@bge%!&0Bnv+pu_fLlQ*WZqr{gt8pPm%rqy7&HV1=9)r$S?(hAeSMD$Syxd74q}vQKVd-iBKaB2-7{MWz-_ zgL!X=*i0$4y5+VeoWc3M&0Q39+ZLPGe&N%yd^uPTT@7mV3sW|kP<_&*vi z3UuH$Bcv&)6ZkbVXbXB)*3w$ z$6BgKuOp!yb)8jlTs$rQtB;{su?S$u7D)pJJe8*G@bGr=@2 zK=hoDU}#a+TRXORb!1AxoR~F3Dwqm1Sgiu=GFl*!jEt@kS;!=a5RZrt!u$H^c+oan zM0pLmtmU!rICev;@@(o<#--rAMmIo?9I^>_{SKY*tWBF~#XmVeM{>~~hxqh?gB9>X z<6rH4gR;qRMYyB~JEQlzhm85w@HJkJY_VZy05~j1{W*HL(G5vSAFxn~aes^d#~o{% zPd!QgdB-;XZ8OzhQ;PrDg7Pkg))q!W)&@@h6N&%agUYM2pH6{1QXud>sMxkeY=W&= z< zq*gr_Rd2(o;&bA(Tn=;2S;lgGEa4Oq!2$?n)iVw?U4wo?z+9`X6R0xw0IxSrATww& znWIg|OF1ZJq`YsPs?l%JRpEn#6z5~M(PaoeIl>Q&pBTehy?xsbjtBW|tEmn?hOtG{ z7w!tx1A{ws?cFJww&XcR$+D4WJTsWu1`HN>#TWNJsY|WXl<_)nW4oDDg`kbh^twwEHZ4PocE8D~Vd_v@pC`yiD9EDLr}D z_fz|!4i7?4wHBa)b&(bDK)FzxM-(UnaaMem&tdf~j_RwrNe}(%NV~{y(v6Ytj51)3 zKatpRyXBZ;Ogn8k&;XDAA^}JUnB(EKrP1YbkqAR|Ll<(x@xd=xhmR=!CN@TM;br$QY<-`Gs;@YmEsbZ@!Mc-;+-cLyZeP0lV(}nwm6TNSzLkX z=;e6}?H=OJtUEbhWaod&%Ont0>m)&T(3G8agT8}(6RYBHCdZe@4BmW!)P%sb@nZ!Y zT?dP=f+j=iD^Pt$F>v&A1ZZs_alkWKs}V?~La!J9V>D@ikS@INbN>td?f(C(Pv-yZ z|3B?Q{?qP{?zH_ZECrc8)jnB>C$WXb15m(|kl=`ticq6x^-dF9792v(hqjl=-GT5Z zN0F$C#Z#(G8|pVs?@qjdYkIi{my9K z`<_DKuip&P+Jyv_c$vMl8-AY)8t6Pct4Au5gA4+;6(a6MEI1sRfr}MRdM_RMlTcRW zrWqXkyq@^~w&d{FX3PJ1KmRd96}2$2{{JSZjY)rsun&E#Y7vO?^P^sY5KU(Ug@=FU zr=?%hjvhh!>hp9?1jRG8W3QBsV8$OeA5TL*v(rJ=1$r=LJl4~5j-MadPWRS7KOV2q z_`z`~Q4^J6b1S!b(NzxB73Jf`5Ky_5FGC5{P?9fMF=wB8;f3~F*SBH&;0&%7-+O4? znxb?PR7G)I=jh#F#fCjvQm=kPr5Kdbwk)qONY5~GuNq9V(nN4g$@IB!oc&<6E~&bB zA}Pdwuba)$NIt{LuAyl@@w{+TOfo2+=$uxxY;##%-mk`}3CoPh{i;{q_6!`Xf2>ns znOAFui`{0SZsRkz6TOS*F@54-ilRSkB-KFEck>mx(?elzrOel5hcgTmt?xu@r^`kHPBB&2ecEKTs~O`#yHyppO|-D*CKOQqm7R z=`!$(X4NgdgLZc!J917v@oAnOQU%@#TJ^Khr#yfKPGuf*KVih8C0YXf1eOZJI|g&% zgYr|4Y7U_b8+HS;ZupouVxTIU%qO3&Yk063uFm7d}J`|rY(|oh>FN?X*b#&p`=+W>l$!z1q zLcG0pDgK?BqD$@B4dW3n%5B6<|Dx3?N`luJfITN-0a>OaY{c=s0;A@2`wrlX@>r1z z85%D@^o}5443016q%lg}H9d-m11?`-L}xtyv*V`&>>Q!u^VA6Z+syR;Dk}cdCEA#z z^M`IGU}HJiRwD)BNl_RHMEs|*78+8Sa3Hb-wnPx(kH}S5#T4pxQ@5rqvm-c6y2!;p zL}*QH7FUA68$VSa)10#1CexVeH@qauj>L}cAz;pG*(NYW&Qpi{gpNL5TV*?8orwIk2C%?S%_be1G1ly8_6BUxeBNxJ zb4}^R_B;?iLk=w%la88tKU#Ii*kQ$o%|GfGvpJ%vqzzhS7hSI0W+D2dBO-S)~pd%2@Egz z{l0GW7npAb2`ipS_)z{u+*!()?RoqK9b&T0Jhk31=IXl9__qZ^$oG@M@>=08cH2vY zc+BFXb-w8+lNLS6#GN{8N?55dL`pn*zaDlwtJ=07*2Al6ZK0rj!qP0RFX0zIe=jvMRziOz!MzpF+Tr41r)aaeje51Anlx05zuoX1h z+=0-kXmh48TaV9tA%%Iu@gs`K2>*bN8fDJU@RRn>5_}T=Ks|KCojf-T)qEQV>1)A8 z9;BC5o%PC6MIE8`6+JeJxjC8kI3Ca4;r4@)o_$s}_leRh&xU-+d``TjOO}m2-|K|( za56*xstdtJk76SqpA_otbW0IuCt}b=@>!@s3oo+6WEUt->B1?n`q~Cq4z1o|t)eW5 zT_nkO?WM~MxSLwf*_twpTTq%|z0l`4b^0S3@}0`{gLe1JhRl;?s{)D|?*DAa*I@RX zD)@{BroW8_vj0-W{AcoFqGsW2{txUJ9Je6}B7i#hlfh>0y;K83CulYpRX}P{l+b`+ ziLwoAMkLZxU2I`_DGSZJk=D}>kCIcgXj&8_gu|@s;&sw~|LkoKt()VX=2VkhV+mz= zybwQu|LC!hC63ABMbYcNk*=VWEcT2-P6{RDwB`A%h1~``&rHs+DsqB=C7ulOtZl(R zP87WD!$+;%ShE_Ve=F8DajIoIMf71batdAFKhU-^DqT@e#By^k9tBES6)=Y1a~@CF zIh?BZHZ%+6&5~I^bNrjb_NIx@K6Pz-begztvcR}=Yc4VE{55SU@duaeYE}D0)6Fig zGxm4hrm`VaN+Dv~auOR`mv11!q=h`G)6xK^)oL^QFYfC`st#Ts^2 z-g;_B@8spi5zF+NV4G9wSusdKJaANhgg*d%B2L{S$LD3u@VOxWJ*O1*K*%O2P(7q+aACwsnae%=e8?ph1aeR@*%-Niy4|t@GY;P&yhcQ43 zCfJ|}o`%uU^r_a@pba%AS4U&doZQ>LK6*XXK0o*7a zx)=e-a?ed0j5kfG6mCS-tDZq|or_wC2Kslor)dsB}rw;&8}uObcuK zrA*yk&NGc%tg0saJ$QIPvX0^wWUTVs&6WzKq_q;2&2%4|Gg9Lr6-NfNc7 z%S8jUmF-bm`*bFqQrSfA`X=uZgv)HXsZ2FXai5LUeo;5bvKrFSUM>n< z$Z8RzXJZx%0ispA_EN}gIYbs5xksB5P=vIE5 z7g}5AON-edr*{`{N;Z0-j+{~zW z85M4Kl@-o*SsZ?G6&Mr&6?t~Ht#Vn={j}KDu*BbVLWOAeQGYGD;W2Qjg!3zV@PrlH zYW>91^&FaxB%(TOwW)Ua35Dqq8jNoTbQ?4E@IksW1y3*9PEzEJ-t3Z|pgO4r7xFr#f* zm(!|YJ^k|Qp4_DGaFe6_0-G5$_ce**0iv2k^eN}%P)_$vC<-0?&GMFY^d=4OuW;bs zgACE2GF*Xv2-mlc2Vc)U=*L-NdOf`ev)fn^h?-VP9VeS;)@D?q{y=4rXR9N;VX`O5;&xqFhpuO4%d9XYj|<6XvJ zHjj7W0qUKA&!1uT>!5G7Y`g;=-<DL+{AygF|TuBLAQEyKc^6AThOkSdn4@s8!#vu%m(~^IuOf5cI5Dox_NCFJa{U z1+;TtEw0nHj)`cA1FgJi?OhPmaI+SMFBv^!3mpZz;D`oZg>G?(N13fmM#;?`ju#3+u2b2O>{KX*t`ZOfa-Zv-4C)7uzzYXJW4i(?;Vz@)|%@KyrBhhlJeD&(6}xz3GE#Z8BJe-^vcbQ#kg%EIP*Em zDuMbY_E2W-U&b-%_9;a3WZoHoi+K}{I|{pPe*fbj&(TuLQUAE}A^>1myFru23u_1YvH}I3u7yZlm;c9)r^8 z0uI=Txi-^83>6`|ifR+4hH=aDauq9!d$L8$mHJ|f*76Ek9L*>V&GCIxvijhsU~VDm z=3?7Y$$?*Rq$OOo;e_dnYSO=|mav7ker!2U)~O{nhL^RE4&oJV-y9}u6kYH{a%YVf zG72Nfo&?1jK$1P9V+BD~NE_2|tk3ULo{XjIY>r`Wb5s;!N0e9H*lm!vRB(!rEZ3kC zS>KtNgHJegvNLGQ%@Zu4;n|95d?N+a@lpL!!_P_8q&>+LU@5jxgRi z(>Eq#LuR$Wt-ts^#8iGyf= zWhBRVggeO&1G@39f_{q&$%cuxsNf)q{QibD=F#TXes2nwscg6i_g z%=^(hOH>c4)Zbz+c;Zn^9k~iJ!`Kjen)v{uUPf5#gQpR~MB$x~&qg#DqthS#&g=>G zC(8rr{X1`pZ>KZ# zZ27-{b@0>O?XF9-gi38!YO5}2;{Z|Z=QNVu0k@5!`Tv#Ut+J%BVlzB zGSWIjz2PH$MSyb-ZXyJ)x!r%5@^5b7KAg$E&22lnKP4>Jg+*S6Bf=r(q$L*G2@K7T z;odgXj1%Syvo5*<#)fhv>=OUo4ieliIy_d!g*0+opr2TYnrErg z-iVniEP=&@{4P+4xh9|#XCp9)5&RW*XMNwBp&=0cchnwV-W_O=n{H`~o&lYAR*+|3 zUsu@9JXzS!Kv(Em$T#GE2as0`!f4t{Jpj8%;D=bhgJkUcpX~rPRU`Z-f2 zTvb&R174(fQ{ke13uTw{|De9$p~QD6E?TF3Lb4iy9Jrc&=>__rKac@Xd$Y9c&YN`)WFO37DGS2Yb0Fs8N?*c+g3YHE$`%kp({kLcv{Vyvf;N)cD_|G+~(tvT-8c5H%(&{c}xnvD(?jBq!3Ze)Z7E@yw8vqdNgk=uMk?0 zCecVl*`bVzQz>j#L?%to#ahsIQzDltG!X?kKE4b^B(i)N-EK>LH>L!v(UXCS1g(^VFq*P zMEDC`;bK`i)2PPj=z*IOUCDtLO-7j+$2zQsmVQa`xh*z4qSQRkFs(-uZo#x>@Aj9i zAfZNKtd4aJ+Wf3Rjci~QvwF&iYOO6LEyI#!-P#bWD@9mko?+F`K}mt|3A|&i>Xh#xE)cc`)&`>KiH&{x@3VQqLU%Di zLNh8BqoVGsn6ANJ>XyfV%~$2WBPTJEBRK7)073+>fR&eccmt};z{v_RN^FFYEuK+u zgN&sHI@*makN4e~`<6&5IP&*dp!3k-2e%9INJ2^-F@G&fC{`v22=LFBu!}b~HagU@ zqT)ctkO-Mq`dQzCt*owk*l`6iGK3|&b2sqFG%W+yu%%Q9q2%zdfQ2fb69n>7BuBwE zd24-lk6fe6V}f;W^bb2Jb2V(OO?mlIfq|idP{{XHB;}44C4a1?^wk;F=FA2yze-!# zmovQjNQYQ>(O%){hRPr{zD;ERdg{jzK&pJ9#PLei2rSRDo{A~bTv9uDT%5*taJz-=Ido%h-oS?^OM;^o9RJq0Q2A%EBo|yZZ%Y+c3{RmZaU2KCLqZ1h2Ou2YP%`vs<-jLJe zNsJ)|hY6FvE3Q#7#N|Q7X}WTzLVVuef*b71OL%$OO{6VfEhg7Q!&siO?@5EtSi5Ym z)^dFdbN`|LReN^Vb2NkU8Zt4Qz<@#b$eg6dM}yRi)DT?Z>e`(ffTwE&Y&CLhrD_mD zoKGjydq{H({eyML-enS9w4vgg@jG|Ziyoc0!iKxCvAzp8H|Fx7ezg+zDXQ=|OYN2W z_ZA+(dn#n~G%-ir7HrWD`%mk#V0My@ny-h>6J|Wf-xpwV{BUM$U3p;rdgjNqHH`|4 zFl`^S}T7JW@^+f5i_KJ4a1oHAhwRhUXXvfTigp3FAG2r6hV_iN3 zzkwd1cxL0rD6}t_7{}VLjpb7VLhiYLrbN}!vCza{NCu+gXUK>10?4r^;F2U3VTugN$3qcU3>O=Zhe;Sy+cz3FRwr&&PZ~z z(v=JZ)go!H?}-h0EWFU2@7BkiZ+6t;a{|M6zD11tn@q^46?$|b!FTeXXk?JrvXs-p za&v+_asy5^I5)1^@e~%uiTDCbT~~!4>GJs&P_A@$y<^N*7^UuE9j79iyLIES>a#hU zU!2P<3+%9U8WQcr3oXJh-of9K$iAa^I%y(fVE97v^`aNe+nu3fhR00W-q&d@q0_Y| zaretOcb+Ch?$-t-?+VbpV*2LQuXbEoJyuQHKGgB>yK#oRote1BKKOp=iYTC)UCz>V z#c0(W@B*88g0&*o+_qROlJzRoptGX|^fg~GJJ%y=T6vnHi&XDH9IGw0qk_~3-9JacO7o;8i#8BW%)bb&0vYVKGk zVGh2{tG`sad#3h1u)#0CKyxz@EvZcBBT)VAV-Ga67(Vgn1F3U ztXTnU*-cG4s=2G;0YQ`#O(8d~e{2(#;u*O_9jSof9xsUP1aKL6xItR5N|>QazPgQ@ z>0Xmjm9VN#bS1q=JS0W|Pw5Y$M{d-tfPn0TO^3dK>;4JQ(v}67k$4 z@XQ`Bdz;h-p8A?LXaPa;=(2d!??4`HSIuSByx@brnd0kgx%PbHRR?@?ap_x3Gx|N} zO7in_@((WLl|vujdR~-c2T{#TdEuO?hwN>dbECnO2|Lro?U7bay$i+r150hl5_{m5 zWalguS#1NdE*)gMpj-_mf-@QT61mmJ&@v-~!cVeOBHTZLNkDu7tCCI(r)~E$?U<(Q z{$aOOdkbEIqFC5(k{OXPZ6NI;G!vSMc}I!EMGOItTBaMaA=7IiGY z;t%fV>smI>GNa9;OE&T6hI&usn@;jB&}0QdYG88wML(b;Q0PsPtSd^IEUBkgSmt$e z(7sv9({-mHoq<_$LMbtr=zx`MazCb<#7F!h!H{Cb9t>Emk&(W39kKfADFpnUESuwS z0V z4HCPWw0hH$Zo57c5p((<;EVigow$E|Ww!aZc(n#!at8klTJ@hL`Edip+z0PP2H3Sg zSJ%-H4td#{!&MFOQFrR`t!%H|1Fqf_81iA5tIowkV>Yh())Nq5f_ut@V$=@SLpwZn z?`P;m5OQYrb$POm>X08ii?!(s>MCgb*#X(0D`|*&z`Z{F>UVo%uGQ*X1hm`L zB|K$o1j(k5&nIAens!aR)N6S%`|fX|tq+#av_qg>mzMBUtr4V}5M(0HY{A-T=vG6} zRzsYY@EnY}7OOXPmF;t)&ica7`r^+p$Oi8b1HMj<-6IX$$Kf{VIU9Q=>+}?qZZhDk zilMHGk*v~v{uc_c7fszJF|^eJ4r_cDW1Jc)cX%{zJ0maUS{}+}Ub5x;;;BL<`Bfz^ z}~tSU!Wl;KWss!4lRBq(XnI7rC~5EoP2JV~Q3%~0hnsU&?{Zqp;xdU3Ba z+50J!bt!Zm^dtV;t1mfRcJQ|j_Pq$3k`5}!;tF_59XRwG6^bI2RH1OCF3GHGmE*(> ze~UzU&QKoaI>&`m4hnT~PO~C?q!qikz{!Z*>0cmd)v6Jbl125%F5x?^@%Zz!x+qej zQ2d%^IrB#-oD>O2&!=gTRy@b6Kql3A5xhCG2RmWFJB!~2^($*^y=t6lahdr&jkUVP1pgi4(;lWavn_25*hOGPYL_FIU)lmou) z7FMXhHxSE>wl19)uJ-J-Ex69rlw@)#hp!R$DR2}I?_$m|h@e|IMjNdKbTyTHS~#{!Cb)GcJZ4O!EgX1rnRR?^J#8*%M>VVaP-`@K6SOzaiVr#)iwKL6n2UO&H_um59hrP^QwJS> zpvn#bcqiT#rx)g(emwp5F(J20)#IPa&9?1^ZxsAO5f&x^XT%A`mfIU@p5E}pwKGg3 zv1p})qW5IlN{E}4J<-N0{JhMwaj?h|ox4l@)OhQt-kw8}ZJfI1CeuvXzMs)lH%R)4 zu1@!ZLD|W}^q$vdUCe|P*TgY-yT-bSz5)U;j!BHNj`{}!JC+$&C0mAN3j_;>2MfCg z>j{xYMIvf>XtL2A`K3@cdjMO;Q-e})_N{w1gPuy1y0mcmK&l={%IZ!y8{CmxsE{on zq^z_g{rIU*MMZyHxN%($t=dxSo|W$5X@~3%vstxmS14yW$$X;SL>sSBH`Bhf@hR65 zMqm-WPvy);BR9)rEA6=ZcKC|SBBFNqQCf7X58+Y=X_BH3T znnYQD+dBcLq2;oOE`ItHzw)8{=)MLrH{PmqXKg(xr7dw&$GCK_^Hf8rBt}s+PCPHJ zmi{S6`it%hese_CQ2vT+W2|E3H>=s~*z%DHk-A*ns39Goga-K3%F5u}Utl+ls?&w8 z!W!*~@?NE>!RfbnRpQ)lfmRvwQzF_C9c9ZlWY>PX2 z4)1kk4d?hT;%{Rw=FL{rH0!nG1SNrH$SM@;8af7emMGfvcwQ1@%3@`Xc!s;|@p*nX z-`qc)qSwNGa5W0N;GU#-3R0R;Qkd$ZS~6I4Cc(ZbuK2Ne%N#4tcf*nQf%%Stp+uVj zDG%Ppu%qub-lP%gz>eRvg|o)^ZP5>yKPfWl=q2sgTlWY(^5F&-zWY0FpJwrCt+EHW z?2}D0m4B&BtMqgEI2J>d-vPMUtdza+n25?VF8S!N1+$8;4yp4 zuF{$8qipi7(x$f5`l`;LeF$_1o%bcq%W*ewCu~g5Vf%@Lv;7J}2%PM>)x&PdV+1)n zTV;Qkq5uItEG|AQ<4c|_S2HBAdc6COtuoZ$U~l%;xwmW=ej*J}o+HepKtALn3JciS zI_)3bXDTNYg7!>haC;0jc`F{HY~-hYz~J62JFVnk!qmqYw=9?4(BYXy-@P%Gt5DdP zEbPBQfOzKJlc%oO_x$JtLt9y85PiXuncL_ky!$$DdC|+c@=_!B@}tU z;1Sb+%Q-BL?{i6T`B+`Ttj{K!>s>P#I?m3#xz?Sw{QCY--3^5K*{iY??v7Q6l^Kql zYUscgq050W=*>YO3SLkBRb(cR6C4ApquPisLM_>HknBLm>=fEqUGpM{vTB>lm@5bj zToVlt(IIeHs7rekbF`8?P?voMX+br{lsOW3-UgtV<}y9FnM0eps?PW^cQoHBLrpRr zr#EYV6dq_tN6wUOtQ9#d={MV?Bj4P?*{Q&mUMoSZTxxQ?ayzcVYzHYqmXwyo!M_Td zfLhnIkA+5Ws}YI69*Js+nG_#)7nLax{F;0ROhh3wb~^-SR6FlAe8TzW47@GLZ4TK` zBs-Q$g@FNUuBK0kiQAU70OOpy286-s5SPSW4VyE~NzZ9Mf4l9O*tD#b-Q~yE zHac|7yxc`7EHyjXE;CA{Ycy7MCoB0ywRxJf9~_Fj8Du|vN&5SN=W1ou*%-bSG3hk> ze5cc2>)40uidVuaeZj`FbMPsN;}{>v!}uAS&YNC4CihDseJx1$u?5_>z0|`b0g0V73F6KaAB=d!U>6eCK8y`hWe3UzHsMt15ut zLL-k<+;ad0ABszX!p~ZEea0`(ySdHqSz=@1l5Bl{do=l)@0Zi04g_)2@}ITLZ3q@`9lv_YV3 z!Znr5^XRZd`WSjRLIp8y#ky)ziIucX1PBTD9FU;GI}QvnR!N18%#&>gN-nM6v%P7X zyrc*AdR@0rgP)Vjb@yEs3lX#(vaH-+;mSENV=644x;HlL-@)>~N-C1(cJ@yHm>>^P zkd^+NAopD@j&Aa6&_ci;*vb1nxV4-x9R_ACjSY6y0B5{Zvo29bbzALu3GrAk02O$J zcv;hKfA0&+oTGn1@TJOOZ>MK4FET*Dn$A@aIOrzwMWm|eM^SiZ$V6gzHP|!zP@$Su zQaWSjsE{@gqzk%boTy7Vi*|d$QX!UZvRlAzE`wU-Z^?bD($ecg;C_C(z0cSOlaa(Q zoS65xaplT3dt0>9k=>jC z;yat;4rvQZVIBlxu7cDtK6&O2qLPUcq|_V%oZ^$J?VPQ{{>s9NAP>3(U7WxUkO$>J zg|n)y%0j%mx!GaHZpPEg#UjBMvW}mG(M2pu3=6dt#$+IH<)aS0OGQz3vaKSDutEfy4J{QV&cu#&ZxetthTfhC)IExUYSmH(aNA=H`F>7t@JeQ7p zYjEH0HE8ZNQ;9A|9txka%Q}yZE2iWk@*uO9&%Cg@1CRU-&NZ~UUU%KR_!qjegcVh< z&5{s(T;hC%8`D*;-wIyV10!7SYcZ&0L&8X)9kSmNG2IgO-rAULk?de`uLp}9b&E&L zrw`HM#S2ZEUSn*+UBwreeZRh>Lz@}E%freT5B1QRVOGE#>tZsiqHUE1yl607!cJk? z=^=hIbek7UQxgTStVASQaYJL6er43*=WU-@2RUX+m#)?5JcYbdtF{3QGDb20^v36| ze^_p^to5kM%odwVMs8ytamv8m@w|x`kxq#|+ILF)g!24gcx@7o@Z2eFUpX(@|cJ1A`dlBl9;8CCQ)KR8K`lt2*8ImQZ~wZMj3Z^3-Y_X)sa9 z*e9*(>kXE-gka2@Z0<+wc#;Ao4%HeA9D$w=dPp#jWR(-aVFh%YvM81 z!N?LP8snQ?4@(mpiU#tQSI7QOHjeOKr7A z610%0EKX)h_7gi<(c{m@DPiZJuki^x_J1pr6a2f!`B&gJ=|9FhU1g-9`xsC&e=JyN z*Kj)?_BH02BZ6TN|AsPxZ6qW89vkYs4aAd}I@@UPalzO>?$cGx{tcT9#P}!F(}Ebq zLf%4zf=*&Amksw7YVhEOgsrc2dIB_pw>c#w%?b8~=p6WYKI7;WQGF>CVdAjfs^xa) zb#R&ej#erl$381QpypLsp;Fl=*l-9+g0`RZ4R+4u(DG(4FI%<#&vCl!H_Eom&%ZJD z`T5TnOZ@NtsEDbvv6JQha^+~$gz?c<#rnu0x17CgvaaX*-6ja(T`1XB$Edb6TGN0> z$FfeCw`8ku9>2;)B0DM@xGLcDjgpX0a7{rGF_a+>=a!LN9u1TrH$Myk5mCkG+ZKra z_(i6-vvb16?7Pe9;%Dmkr@dnL;)l zi#{NV^Uu_gsGCwKcAZogqft1h1hR%}LVx#aluv=M4F|1dD&*zq73B4ru$yOq9wB81 z=dl=N)2RSH)&VK@I?euO%(%z{cXK=C;u>Rfm#J%tUjA~J;_05?at5V{atNUno8M`d z>NR!hEx6h{ed;YyKihG~+~@i5^%|z@CdG~?@^I}DE&DJ9L&qa->UAX~MaLtp`Y{@% z+q6W2%1S2Yfy#)5Ml3}b8}&4qTOf?9Yq=0MhotFk2A6*gvatJs0Re}-4m zwZzk#97&qJJWWJMF@rl%MI<`9SQ0`@1OFU01R^l&OxLz^J&#a$#Nfq&7GE_9R5)5v z$eRgYo%p#bUcrWv=m&FIU(Jw{;>jwEo;=~KfjBKIc8dG`C`s_{jKhwTQ3+?FnsGSM zSBuL`k<!udsZ!5^*CRztSsQ|{!soza@Wu5Ji&KwZ{Nh)o; zUw>GL7g+|S=7Xt*I9KvjxyXvB?lmBf4$MZ35CUr4+urMGW;wKdPR~XtInDLtY4dK) z#G0tAQne@6lcEsQOs+wBPZI8f9ZXAY?uXrACMtA2L2q46qdZitt9JY*Z|1k6o^cH~ zlSDk$Slp;KEE)a8O}Hcus&q3eJhzuQ)8S?=SwuN%H;{B!4vlUqC~*!-E-M5w4pH5$ z%_gDsI8#-`_gwVEI4$LXaVoDTM~|v9vh0p@#58S>8Z?Q7<_7VB@NYG#ok71vY8&6$ z@d_^@aq0Rd_RaP)IvH5bCo0CjpR#H_;Itk6k} z+Z`yka<*WzTa{ZM<@&mqyw=3q4KbsSj5^{{l#nhSGL%J9aFjLQ3Kg;=MZJ)XTXYD1 zQ9)50g_{mjsG9wAOq|)3-c2+HR$1V%PDM@iI)%52^tW2J_Kwy}2T3T9}E-D8D7KsGG$4Wr);mMO3^ZsY|YMUwKPZU`GEr`8({$6`Lz1vk*?y zZzfy}d^POb&Jci$8r?^giH7LOY(Q3?amgNKD5_s zA{{D0Q7IuDxdaD1yTnWR*RgPa=|_Z(cn1`J_tfI>RT(4EaWP2?Hb3c>bo&=%yJQFI zjeQe0sAi_n1T^XKk;ayg){>lSyXEv1Y_6%rmW2~t!9}s~BnEcIGWO*jd94V4?%7U5 zk#N|ta1ARFA#S;XaU0e7-!R1j^hOL>_-AV>sMpz#`FGB+6jRr_NfX;@=H}tej$bze z2NQ8ZXZyhhiM%Oa>d4VZ$+;9&w2mG?*D)T`@ACissA z=(^i*8b|QsJZp8#R2M3KsiQi?_bJf+;9$l|_A4dGvkfD7?OM@M1 zCmq8{G7_g4C!0~R#Jsy{QG>ZB^~{S!rtz&&Lwr~5izr7(_L0eKOH7WBqdkT@g1lRU zU66)57q9X;(oLe@Ko%3?y8LeVh(WQDtQSD&AKQ-<=gq1uiqTv{}t_fIs!E#=_~Vh z_Jo_!8Uq^{45VKRU)e<-=`Q(glv;I$4@u5I_ehftC0Ak)uJI*PSVZDaUFh?Skt~R- zFW7@yz%C~ef$We1eO82`!kj0?4m(_0HrZ9$=HiDX*F+^>2|EP#i6QL}Etymy64 zVPYIzStR2yLm{y%T{p2zad7@M8)+dD9XHu;)^@`RzsXzjwQwUboC02#M;IH+*BPSn zppR@t=bS&RAj09?4@4MdPT1tKpPWwufQ9_E7zyJ8|8yF!8z6(Id4_oW%y~?Tj%sz6 zDp^jOZnh$qz8WI!a;D`nHyT!a=e^Z%Q50NT{pC~=MI+5{rwII~72d!^5@%QlYzU-$ z2I3kxuxmb7AB6?$;h|O4kR8YAi%li=5;4GFi!jD)DOw8~xBrq*!`^oe6#J{iy7%M$ zb7ln?DVzto9FO?8Z+7N0u;YQ~&bagt*5&N#L!1u|S(B&-KOZ4(m)2q_YV^R>SO~8= z=1S%4nRA>JD~g7W#X1f7i4MSZUiajspdr29F5rWxhSy=~{%Hh2C=LTTVC)=*za6SB zIkxx{uz2*?Z8(|<18YQ^3wn*zM^DkJKBI#aUEO4G3K~PRHfxq8LvKep)!G_RS z0W7UamTu`oT#mf6m=LCgD@yxHTIsaq;UlG@i_u7xL(aAohQYTpx=+kTCkCtaJHgUe zukL_(5l3^6nWYUVXF-;+zQhfBLr8sIyG6BsS=QN`62ppPF#$ajW@4=|7P6&`F~Zq6 z_Ct|lDRs)efa3W|o`~UVhdwJhsk#CIG(WIOBDotD66`4l$1<{n4HosXlL|9^>(AJI z+HYU1g;2B&r(-W(aQW!rW$LR8$YOp;w-g^^Shyqz_RH}qFIbl)U*RP6cP7GeGONqv zw@$;*k<-uP1V82jwJL?WJfDsmy_6Q8$@zLCeV{KiBbv8MKn>PeB6yA?E) z`Tt9#QXxF#biey+YlFO~opB_jhMq1F5_gI(8cgL7@Q=@jgb^lQr?~~^^ zA%9Y1Z}|T&D-5GQ=J@`iak%g77x}=!!G*!)+`!%3z~RNfyKd)^Q|5Q}^2YIYp~v%A4CQ*226s7 zB|`e0<}{ntRJ<)L%Rovq2w-m-7X@!4>)>V(?A(q#e_wR7jYeH+}- zu{UkSv@{n*7|)vu)nXM1E1?^`EwN1!@0VjvDPKY-8=%H#5bSNba zQH<^V+VW(~MYmRKj~!4b^Kw^xoiWtpcYsgE4%eY)?!sL7o*AX>T}>$$n3wf#&>dLj z`<1cznwgwo{l-btKHeOR4SeM!-2Yd5L&saLR+WhHiNH1vPU}0$IYHR9bY5Y$HASqG%)7&+A2x#Y(7K{!#a#Fb z-oIhRe|DF+{qr35iTm6`=Ps+cGtZ&mppHDCtNMhTcDW&B*4D*T|BCV#TE{C6b4=l3 zL%fjZ9>j3D{#Edj*}dFOpEKT6e)=rHj2q=32*xKwoN5^N8n zJn&FMcU5wj)s|j`w;USdpY-S>3XjT3u>1ZH%r^I;F68~S!~B^yNUFLu(#~(EVn7Qk zuXJxJ;_eI?c`!M77W5EFD}EhC~VeK{<&WE-sZ3dO+9W1X({+ zfntC?bdx0pr~YKWa`?*gEfhHeg(K2%i}&Ehw-X4r#%7vD196^2inyWOIZ?44*Jd_e zm?7UD_mtOLjtzWWHa*VRo2;TO)s%ta1kOM6U-9FLGv-~9yO{)Q zfI?m{sv)>4`hl`~$)8h!q=HWM5uYH~_O~#}^Y38tUtTRqyFXo8{)A3TJM;g`5H>4r zL<&R@5%9q@W)Ve>z1y|H5)lMREYi^;K>Ay7qEj}#u)k2;%WQt}8ujvzlq;htm{%eCz3W4>#VpX6)leu&vtx@qX%b7%EGiaVMyRE0e2H_#CJ?t<;w z7xOx+vLYBmE&Elk|tWTW@k)>bE2UDyQ*KUB)S@Sf zIJlF}viOuX4axA+!wpYBmJts~8MG&iVitm*JiNC5^!}v=ovn=h#MHyTrBMHM9Q5BU z?teb0`MaQYD7g*~vW52#L%EYQSt|nGYbhko^eqDgh5swa^fds3sI0J2S5rv-E zn{eL7rLejMS9kw9NL68WY*c9t1qe8U2Hg}&+0iv2qDg6k??PiN3A9;Lr47CBhAiy59a1f*zE%v zInYd@(q{T@y!c`fJNIT6J^q@-?iMjM2VDfEF&;#&U6H#WE3-T*WK*3nClDq}D27w+ zwDS&|LfBX^h3Na$yvtb>k_z8cO=C1hT{$sfXiEX^c7b5FJ% z;KC-#*YX3%u?eA)m8dvqK=iDO;)E67d}%wJl}l4y{=;`MNiP|l`8Ux5afxZ!WT`u5 zjeXScSQcr6xDA*FaE)Q)&Pop(BP{D_b@UW5Sk`AN+BhHneNXp3X2xq7qEMcR26GXu zhV)Z*(oHq_xRr8Mz5w6ekd;u4l47lhltc+Htn!k9#FOrI@&TP8H0T7P05Bc0XKRy* z;c&~-XM$7kWxF9KJEUPRp0;T2T81Fd>LJNle!2tM58yw}s(I-l5E`HBmHcnbwf@!L z@IS8E|3T#F@3NnWj2NdLnD_gTW zYLed?(g2~8lubjbVzoRpt-^lL9{~_tKEd%=fUfHT%=0AN1=sn3%iM+IjrPM#?}zi_ z3;*{=lpf+HmOTgVXjnB>pEyC_XxPc^m>m0}uS z>8~&rbneyWQgiM_JyEsRe9YFqn{HOoKjjpUWZUOh-?|z=lZ}FohoLSs^rTmK5481& zzK6HTZlDDSF=qd05Pin6%a9<4Yqzd^cyObAnqG7K$z<9>H@D1GG_eU{JnL`eUGc)Y z>38wdpE>(TrM6-(YWalo1$YhwcKFoI3WvL0Z4BX+D%J{f7InCUP235|N7;=w=Y|tN zoeO9e{N8ND5Xzc;O2-O@>aMsoJK~7znw8#@WGl_)YaJV*>CE`O{;F- z-FGdvN}(ko>0DaTByi3#q;#6kxMFDtY8^!;zpKpMiTBQ}x}A2kWINZ)*3#hCJa>y- zl_)~_oie!#B{lY_JXLNt>(ws+Vr5dLa~~ESDL?^*kF)3|W+~<4B5#3hy~{?^-S+%O z#=};}m=^CMx5^mYe#Vwr@8#qXeDj!cM>*i8`nE#K^4wVy%Sm}qIL4dQVVl$uRv9@J z=P(^dI+yGaU9+d1>pGRsbwtq+?AkG@s65;Gbj+4LXy9LYD1u|Q`WhujO7RR^eeC?{+xDRvl`zwN_D@G>&!0X7KvR= zFAxI4@Nx-b{FL9;lExA1f!VHZpjRa?H;X=hNHn9TZ~;R6V%6|W%! zT8*#3l3-IokC6d8q|xjWN63bLfj2E!4pXyQSryKO;i`KiXr4+O^0B8kWr$u89BmaCIeRAXjEvfyz9b<(;*}oD zlejXMm|3&v6YTsH#pRr@5x}(^HAh;6@@6ah`ob6CLy$mMu@8rxQi5;jw?L?0MB$jlNrRTm4!GO_`u`j9FHo^@OqvF67vZ8^} zGfUrN!q+cHdmC1K4ESBq&tuUIPO7{e$2$2;9f*B7kptA5PrdM5yz!20e!RW|_ptgT z42Z5t5Jo>FJbqzMXbGv)Nk&_fGeV`I@(JUHR@;4~3WZUW-30hD$)xlF~;? z4b7)`NhQ4I273-!KJvYMMNXOfl5X2F#$I!6Myri%rK{Atuc<)`L(>dX7d=?ZkP<+? z-;9-S-PDUkUe>px?Pnd!jc~`CAUCVpo|m!OMzZJqz;Jv9lVV5|1$O>j-7zSmqttsK z8q82pru8_$uf+P(D$jO;O?Knc@8f;IQMCoXRyC*{PQ0Ngb6hS*-b5i-^@K8qr<_#Q z&hO&pjz|@S!ol^cAysI5*HLBLEwwD1=ts%=SWmRGuLfVnca-~4n!=A`swp9b215#A zhIn^>i%Y$z@HPvrwnOf{`}Zfw5B;Pck;r|asT+0~v6#heyQlciUWIqh!v4bljrtrsX9Z(L&iexK6wP+FVK5PXnpm;t0BV)HU_1FDWWM z0IvQJ!LSDx_Z%iVDg-p2DkB_yg$ez0lR>U|S;O0*GJRlx%!Q2_JFML}bZoOu>8msv z_mb)*ns8Gr<7*QYHVcg|p{t?(!DX^92O@2-OhyEj&jbU3m5!yC97VDrDTOtgtKN+S zgm1Yi&Q8056gV=V2vL}irBA(YT~pcY~r z#|+wZ4V^qja=TSJz2SNG@mU*ti>&Sy34)9|=1{13XddFRS+O^Bw=6+2i?8|@EBEEL z2AsTZ%MKxm>RCGwq-UU!J|vC2M;zkrX>CwVtznQ;QIYU8!gFgNW`SMT4j|BWcFDfq z>_O7O6j?TaY{vLTm~LVvA3)rXFuTz={Ja6MhqetDug zf-S}sAeiH`qFQ)FFen6q(S@CN#%#X*UKz+KSW*${3~8pi=iJdHmnA9O#kS1&aY~C= zCol!iw>W(G@+d?=?KN%hDLv6t3Sm8(H+0ssGKX{KWz$DX+_Yx(vP-(`C7p66aDr21yMz5ml@?%mE5Dg^fN% zwqTJ9fTaOqsiT4vsy`Xp9-Ui_ZWdeF60$G74_ znbbJ(y(xE~hLvMjW6WU1V8Rp&-L89yN7G$xseEhb2QMN1bMJw zNNj9YU>5?gtqf5}u$rpDT)k?so+-lBezAwwU+RxaTFft%X*9b_6s{?{wDl|4upnSc z3HpdQIrA3nXPCH9PIPfZQPxdI3M1h`7OXw6Xw(fy%&tGqnUOYNQ^Yy&`LSaYz+^=g zD(2P0Mx&}Z@tAK3R#d)PU~RC!=gqQlb4j!ukgX`frvu`C!S!)e3;}eDj|Cl zKALoPtyI6+vewe2tLt2Oh*A)cmZzA<@>|o9%oRl_NTml38}GnM^gC# zK98Op{~wtWF-x1z?gO^?9lxO8+hh}tBBa+HnNwSGGKgf788+ti;h|T!|KV8<_Z-<6J zy;Pn%qTol?D5J-Wx|-z-elZ;!XjnB+D6cEfZ<_DnKbqjYdR1 z_)((YU2LVLdL_!H-L?YL^4xBz5WlMH^zx~au%e+S#O38w!>r5kv8SUAvZj|bToYUU zAddzXbT9LFE%vh-%S9qM5=cw+y3`{jLHVU9lm@*ebw)}oXAx29BWz-E;ArCE*Q&jl zx!gfv)K`@KfHtL2$Zp*ouWjmOhZgU{mA1h141XUT3|!WblI4U#)39g`U$PB-O4Aw9 z2>Tj8893;npZr}L-qYA@;=5WKz%`{N;6yWu0xQ6%;L?@qXJ^RmpyxAOzN*7C#v}oT z=LMaK`ORq*3%lhe@p8cpdgW?Mz0TOKDK^4~@|Z5Mgg;Ny`Wc^_j?R=K_>*SFTbci# z%f0;{L_`sst9L zd8wkdRks&6-9}5afreqWpGzEv%w@?Q-4Mf3JeSAK=OhcZK*OLCCm7^~Qjw{tUxbRj zAr_BJJp5vSN#Wl4Xjrd>PGLL$=Ly6p8pBRr6m32ezI53dl0^Db*gHj;D`oBlcohZ1 zX42@Ye2ll=ERt+vW}E@wRRhV49Dwh?GT___EseO>N?nOSQh@aHl~WiB>)Aj)xik&4-*w=s!bZnX$Bm`NMc1-EweWd zo<0+hw=%+lSVF)%YdF2!v>HdPL4T?CA+j+E-!k;We|KEb$f3LOw0=s`KO_e{Ld3opT z+RvIGRU!yfK?P4C>U^FwNf{}_C3`_3p~$hDR|Dkg*KOUJ#Xce~o?)K_5(j>l^N)I% zuL^(#zHE_mGkYJdIiI&OFY9-A`$8~8lEGQZluxsYFccWEfq^+o7BW;D2?j;9Q=eN( zNKN~qAz=1=_p@7F(`|yC+Qk>Xl`QIG&~#kW?VM=}scyH0n$7^wY1xilh8UmF^j)`V zn}yLs_t7fqb=3^!(sHv-PU|DF>07gxudTC`p^Gx14YO3x^`l+4#W?UPNzkUNJ;Yn! z$gQ$+8jc8+B{0!d7Bhyy>b5grqc|hKg%J@nfmtTjri$@{_eqUQhYuUh)Wf0}euD*} zm`XC-ejR5uPw?()p3DH$#7;5}gcP2jZXS)I_u(^oP-a~0BDU@(PIk7w|Pf3t_n&Ky*nMLgWubTgrSg2OQS*K7ozbM zdqrV2t#0vxbZzHQ>NnKfPmN=CxaAKi`xZ@V_fS z|21&w-<>i4563KQZ|7p_@ei-7q|XLxoX-Yp_k`1Q*UI$SvdR^aq5(9Wo3$Njk?-KX zC69cP01m^fq!GI-{oBgvdwCTLq~~dZX#!O*^LlDZ3!heJ{z=YrnFzPBHqn6?!m_b#lw9Lr1KdW zvAttwh4fJMct!h$ZoiTaG+d0&=eZ>12Ppa|wJ|1~yvoEttFwNq=v@M~daRm^8L=ht zPuKL_$s4$)j0_({#f#~qXM>`JzQMIzpcS}7MIZF*vCH$YKoa_RfO3|Gqgx|qM&jXU z1v@wq_kN_B6jMATb&ieSS$I?Og@3sz0dv@1wwdVXGJLTNB^uW(qD1#X#B&X;8?HPk zyoW3*ersq>K$_mlsRH(1Nq3#l48NgL;e6H+eZAgSg!1ou>J1OFjIrBz4g6-z0B!*8 z{FF(r@cp6H>G7_gCb6$l-{Gr0YW5ZJ7ESfX&{)@@6#ejTp%JrR$c~J$7$FDv#c@E3yqFCsVQ4`~fi^h3qA5!x%t%@}d5^;Z&QNsIg#B^7HD3)<8{ABYDf zqoM;9JnYTK2 zxXD>wqHKL8T1I1X$a7Qkt)msNx89W460hG z%LRJSk&}Yse+v~_D+~*VLKx-~7vF3=KU{o)_=a*SVo^X~glLp8O_MmsB%egnz>2`0 z0@_CABK8Zlh{crpM(Rlej~MiD<@a^qG#`fy)W%OfY9aO+vFP$amDcdggUnl$3`RAk z%s2VpN%C!kndHAF20DzsJ0vlVhKI0tY3|qU$%oZumE_Z&&kI>O-^G`j;^iRaduzAK z=$>FVJ-DBFe9bY9auo^Q_CG)bXBFC;9Urid@NMw0?*KXN3z~B8sS5e7uM%s#z9qtf ze;K5QBNww=g3v&uvQnrNdoW*zR&d3lTsIz0v+Z$V znA!Iz$QOui&MJp%j=yZ~B2%_6z7P|apzd&Q_2T-a>hP5R& zk|Ti%lpd*1Dr#KfRuH+7RA1z@s7xe-NGe|;X~Qs-5Y1%f7(Wz-PO_9fr(u^Q=MXdGO}N6e`v- zY1693<55)4Ek>57{>i@^6R?4xVbgz^SWpW`2&n-K;bdt5$)mp`D$~R)1|rHeZ{RGe7MLi?m)2j~^nn%Y|c6B6XfxJqY*Sp0re#av%N zS=v0-TsXHA!Q6#=R*0u4@godVn~ch@7+G!2v9#6hQ%9-e%A0vjy}FsT^GMekI(WYR za3W)Ba}w|hDsNm6{}qGA^=4pI=%Yi$w}50jt8f}6np=&rFcyUAVL|3rx&}Um<8M*C zFS$UKYiODdJg2oLaaL3;V=AM5a9|;Tsbk}-$ic1g3NmO&?qwMwMO~z9$<`(+MqXT! z>9w?0#)f6~V{gjNSG7~Qj+CE4TIrT@$!2G|oQ}StW#=>kh$Lo15v6a~k+Qav3F|VkQHb}5 z=S=108CTuH1p};|Ol^KE^S^oOt|v%ymw5|b8Tn#fQOY(6?TUAGu>%H~%f7|Q?s;8SX-O)8gi0)uungjw~9hYgAvh{>JL9n~gSp0TWm^QQf zUdUANjOI_xoM+A0ee!xaW#u-TGYLnG47GQY#Cwayt-WdrYb-J(-a@ovjFq#c40Ixe ziv`SvdOtT4Rh*zEWsX0rP1bLJz(4^R9W`~y>FqIHq<;m;NC)%TR9G19H0XJ!-|n+bL>g~gl7miq6QQZ$X4ANs9bt01X(K3ozU&W|1+4uO z5Hbi`U!rI;(uKYTA(9INsHqEMzUt-(@D{59Fqb4~eIzRFd|t5E4{O8lP4f)SEj%)r zR!U>ZvCRF$FVdT($e|V9iOJre?#{NVa{N#n+V{&@XJHGOlXn$+sQoM~Z9bfqXyJRO zfu{OnSX)vXPdsL(ji^wb&2#F}p!aBXSXsaGUD@Am4B&j{AWtpw%-I$1XQT*dN=`Mt zUE-K8OUvKygR#trFHQ{rKa=jrEX9gjRt^F2B&#d*ZmQDvp^%ZGHD61WJ;HQL3$o%5 z3wGfPAwb0g6G6s}f2qgwFK|QJun^t{E;|Qz{t&sZF4#eMJW%9a6UDK_XMG?;VcbaHQMiYayi71Kt!{Rn!Kr*wJN~f@N zO6QDZSX;6w#e_P{7)LR?a2+fDyx(vouAU>X;LB!+BtfJp|IrBr3x&rXuO^=S{u6`+ zuOvUc<6;~y`hg(+DA7u-P9dc*t$o&$f*l>wsAz*oP#PM*$0E(#QH*~pHA~JA4vJpiY9pN)Rv|*R`Bm3cU4 z(DirlZk{zNJMh6{D!$Hz&(L=xs_?z?)F!|zG3Fep2R1YH>-GN+Yi}8r_nJHj2X}V} z?(XjH?(XgyBoN%)-QC?GxVvj`cMlL?pPX~%zcah@o_XiG_9I_;?q63|b#-+poijFv zG>|iP9`m{+-c=14uEXRa5T1BO3fQwpK8Iaq$rf0*7tlqc-67^0VrjV;OcgFF3BvI9 z>mMCDy?^X>6`(sOL;AZ8{XfItpL65?5$=>^e+zfsKWyqR*z)oclVg;maHO;l*BxY4 z1%O0EeT5aSrk12|ZO=!}M1N7x=?Lula@-Xl8j}l^QYs>3V@+o~o!1>LUtRHkYOKX6 zi0nXWHVSOTFy!u&3j`NFP@uAm!IYZpLJ6)dkO%8`Qm4A9#(W5R?s>&BRcq%uScPJr zyC@l*$sWxFf?YH&3Oq64nFcdW*R=&7ttB`aK!ukQ(;0P)bEIyP@?6s}#I9l!PCT7W z#ZmI|1Ud^EQ3eQ#3)wN3xXhsKgSDLBr&M_j-Dl3>pT)CHbl~Ej-tr`Dz1a3d`Egqx z%^5;w93yx+6*kbT1C~GLX(}h;qwN#5ZQ+mfZ$caQvgu@~^dz6Aq!XSz&l``O^~zc1 z+f)Kt`m#^vA)hKjYkO)L<(tE~s+rF9F3mAdkGu>ux=OKvhfGNv1rpSY70YG>k}YPV zEL$WLg|dqw58mtz^eG?_6RLjhV<(TOk%R50>>NgErN!sOeQ^>zx#K*9o8CyzRW6QD zbC}!%Qlo@EotC_KQMrIq?RZ&x@6Un6n|%q#+G+Y0X#Fk7I#K`0Jt|s{Tr4yr+XJSh zYpVzMmXK{!FAq)G^|%e@Pd#oSg_?8{&?%MuZNvVbpUGePF4LdgQk07J?{h@&!O3(s z>ZJ-j1Vj?ObHE41PU_X+$?=eciaEhVFggtIE`iIBj>w zn|;G;X_h=n_+%5JBLWBsSiOJxk_UeK>F<2YG-(jP}(ilnhdxIb>j56UdHD;wG z`myO0FeU!_;-dUu^ubYWXBW|FzKzt*Vggr5Fg)X2x64FA2VA(iwxidrLF8c5ocz8V z1Jjg7^;xQXCtTu-ZmXVw5;^g1&@>?ak1~xn&2i@RIQjZ8VvdHxf28r3rQXb?=J3v| z=85Ni*G;7DX$OrwD5?ATe7K|~N7r~RT3ZGR{ix^)bn>3DGTDfN_&s`IhxP5-=Pjh+ z7qxWSv75FF1y`)0gc9RKF~it*7{-^3S`8UEJWE1P9HbJjWsx@v1>)@ko8XJdLp|z`Da|I;uYwHj;QdiCj{b0Q!dz1cTC5j|3ckd?}xhNzoMw zOE5@dyi)%)TvMCM2hu%=N(h4f#|skIFPyq0BVgj0P<2>i3FFL4sJUVgbjt8VGW3f? z$W_@Bc#`8)Is*@38|#5!89q3!JCw^>!Kg*5Unx2U5OdaVHW0=;XA^o>|we8Qi0^v^VR&AtSt$FW3ewEHel^#t1OieLQ!R$xhbM!w^ zPk@mDD+`tLR`dWf`%`4X{X&CWUDA3Ktx6BZ>02M4dR9`a0+?=__x3yL4-2YP8K%a% z`0Lo4b@XtTYIwn{`+ja~)p!_Nk;&ML=#|NnUW<%)W&fbS&HRXFytMd{HlnqvbQyzo zHnVl{9)9HGpD2KtODzS7fQSTl?S4XB=txH-#W2Kh$;ejka{mWR!9tO9OaM4=?BB9A zf`7-7|61SwN18{K`tuhQRn&Jq9Igun+NK~O%?RJ(6J$h%LP#TPAULpfB~wMrImxtm znbV2DK{Op8M?gVDbHqRu$~!9Me--!We3~SYU6+JD`Z=zB_1fLh^=kJ1{B)S}No}V% zoB|m-SS68AF$jyJut%k=WY7=|8kvW)a#xN-wb0;(WpWadU4A+cPTa2|A3aRUq?JE^ zmm>P+xHa7Tx-$Iey3-#+Wuiag>nUhW=KicSmEI8bM(5c4GHPdaK2!;6rr8)MySYXS zyGrI@29IZixc%KRF5fL(jGV(=wf~b!W8L8ujEa`!vP^0)?F_Q?y725yq-nb%x3Ok0 z?gQIiyLEAuh5gX!O%OLI6Ai^(1YCzce`;Y;y9QS^ zIfTple%p6UCxo=0H82cD73V*#gN;?C{67W=l9%3As%? zqQnJ|x!qM)h5OAhdzM%nGSaokB#c4u8m{v%T}>+VhU0z0dp2J)lW9eB)UVQ%1t{z*YiJvjzoYmmMfYo08|&tI@Z zyo7-7B0wN3p@%!O0;@_Q3oKt98ozNJ$5K1u_{&3ALMdsw7403J3Xb}#)R5U{?wZ+WC&^nS4a9*5pCcPZttJ6%X#A7d zTM`XryePI$zbTx$gFx>U_Ir;i+;XgG`5CsDqi{epNTQ)A2#o^wWf3k!NEC;u=Y!(r z$@sut@E!@18o|nudTy-a1j)04I}Z8!?%YAB!yHz8c1Fj$8CQEk->#AaY*vM8 z5Fe9~M!>JHU`Yc+JB0xah}!?pCVNG*BH=9q|}^1Z{ppn0AUT4%Z9R`~Haz zl1lIyc}1hr#M%390lr?vpq=p7Fe0(bCa(!)F0+AZ2#%8Lp62E5mIsD(4E1PtqsO zTvFe`3dz;Yuoq`i!?~KQEt-)ISb(jFmkEiCDb~Gl50(+zg{)0Aq-nne2Xx;GTMTru z%<*_^gt#u`483Vw8DEgoB zRh#+Ns~ex+8z2b5rl2oxf)JbUKf<;nxMo~Wcd8jTrf3nTsG?u&*d|gLdsYFRSXGGq zhWL<$qSjveP2+TyHm>F8h<7*EVmB7jT$dTsuOVR3iaNCQPn*LuM|S7S!X#>kPiP#e}}Y1L(S6`vLNMqFN{LD{)!Qx_vbHv@9YMX}La zOi#*v|IXQ~igh{OaphlTG0vWs$Mn~3BS##PF)DOg++TCl6iXWkzVWGUPnUPbt1AD} zWbMt_j3>cpUo@I9)=@H2R%wxFnKBNorZGl!&%y_K9?^azh z8qTF|4C`F&XJ@xsq9fcr)Gldyq_ld(Wy<^O^N@S6qdBB$?2<1u9{03m9)Xej5gGHH zF!D+pCqO0ldoX%rRl5Uw=_2@A9!LAX&MAK|wQ*z-hu(>Popd<=+(n^tX3`1+`s6Cv zD)mnEu#;X0o0Q1d7+15-X_Zf4g;9o^2s`x0nupYZRP-Hy=N0@d&-=GlKs$REOEXL3 zKR2lUHML1aQyE1N@tuqi!jKR%SAZ@I6KY38Kom^mAoMd3o`4iyhzVqXVanR3`x{kN zyn$sBe-FOtYQritVWkD0r)T@?tLMwgua~zsa$ifE!I|hXvjN7c0U21py>WkW#y|!% z;_=EkJnbxFBzL^t2uMA#+JPB680p)&?phm5?3!hD%oT9qvwF)F zc=k+hJ*o)Aa{i%Q+$uv1OW}(dh1N7*U!Zs=L2tiFHUtz;C z(@Kaqlcc^(uk3%8&M@5fWa@X>kDG8#lAU(aq|!e1qYZkQ+mkaktP4 z@py8rt)5DXZiCH%s9Jfmg*jl!0zPI2-GJ?P%;v&EY(r?%_Ho50Xd7+F5#G*;P9=4ZrFz zwoHZU@%OU4AFzeqYP@JNwPyVL?((igqB!w~M4( zbaj(-Lyoq%79Ast-%{4>zj4uroG7)Th{p5XxG`)9A$CdR2HeZNn1LMU<4i{12V#?C zDc(NhhKhf*n9Nbdlxd)(hnH5vb1 zGx{&zl9H>P+#lZ-!12WKCnHW$)|FonK;>;ntXi*{p$XrDY$sUnQ`3)BG&Vv+i5~+& z{1I|_ei_TfI-h(z&4eWT6%aUQkk3EjUINqMBPyLSl?AW>BsgDf_`JX0p!j+nvoS=@>m;GGIl=JlH=+=;_Je5Gi3 z)D7O2FO+g#O<`o8aVSw(FQRXh^rC_K8AzzbEBxd8pvxmxWmF@1X{-cQoo*)&qGt>~ z396;2{}%Gj4?7(HX#@EScL9H zG~NPEwU|F#d!GXH?CoBzv*wO2+xI~#OO~G|zi$xt{+;VBhIN`*7h+@%EfvyRwv{FR z$nq)n7iE1m{^x*a>*3PpUI>{k+BoLMB`Bz`y);#JZGcc&V(T4XI^?ly9#ZR69KR3v zrl(pmP&tq5B$2Pwl853wD98;N!=yyrI!a7Bgo5hh=hm%yLv{v0v};WiY*DkyCCA~a zuYb&={k`#(M^We+=J3nzO!OHWf1zyp)9bapYd`-gWsE7?*V#T>Bf ziU36t$VlU_RtUWt52sWmO}h|IgJ!BIRM6;(74HWN$Lx}9ba8T$;RDSt@EVNqkQsio z;C#N@;ZvIZo->NbK$G@D^rU&|eD$`;{Uv+!_izfk$Ac$&2XZI_X*duyXY@dplB}qe zbkLTvxG*)@NGx2F@^Zid!x?FRp98=5h_TU7hwj$NjXwOfk)w`dDts3GUJqZhki9-v9KtI4;XMit^(+R={L&UogQ9SDE^>#j3dydwg|u5Upxk zVUwj&*;{tobN5^8<^^RM24qaSsBEsQ6ni7q7gxoG!^OdRGU5!drc8BO-Rif=TID%g zKWf*$!)|71(g;qT#c+Fb^e-&yJ8DykwBppz_ZM-1+g%tm-m>ktnA+j%IKbraY>1&z z{FD|y`l7dC4iASj4HnPiVb>bOXE-O_#*@-nG=J!9uXGQCt`#n4R&Z;vraYc5xs{^Z z+n)szsk4qjsnU0#M`V;gvAiYV8Jk#+pX&%v`_W<94K#16)^IKM;0^898)_j_X(?SO z6;TU??URh>dGD2*@;z1$;PDOsv!}~RvT=3@n=a-@r#aJXYM>v>kbFs1aXUa6ORH8j zNHURsIG?8tn2VkE)sF5iKXBDHM~KFhk$b?uOmSh=uV-ljfatMQTH;Aevg-QMh!^M2 zbw|(j$yoorpHjVwoh93t?P!@o-~B=3p_!c62nKUeH7WUeJisuDS_9u9@hDm&sew~| zoS-RQBUM9xuA*IoQ>Vu$^2E%ca1Rv+Cig-)YgayPs>tS)`nks`FESG?mwj4TGllK1yzn^x zWa-Jjhx9ow8?OR%b0N!cp&>JbGkY(F*PTJo*ukHqi4wU`GIlkDK=fiAG|iLM0$R=? zX)Q*QLBxNjhZVC_91%*+Az@7td4s6;q6akK0W!ZA%B-wQ*uR4$6-FkuN_^&XCV~6~ zFY*R?uaE(-o+kv#fIkXP?js=-UflK#A1KDPAsmps&PUN!NG>UdV4PMIs5FTNU#!50 z^!xTi(zfIG;^v9gib!kFYF>VNpF<2DA_6L5``sC_#cv5M_8xpmV^~`yvHF->KOSqw zyH#GyS%TT;z7f8|M&(7_=Y+~jmtajho;B~;wSzCshLQ(<8-uy`df$2j(*5Z;EV`yH zn;TQ(&ff2eWE@KE5qFDo2omXG?dcuV9F$^Fvp-uG){mQonsNNnL{^mJR@kUoj9Bea z!Z+ypv4J_P_->RLQ(MYD#rE+BIBu@iY3T!M*aytt4HW)6aOC_4IR3R++Ro-EYh9#p zSrqgsYIEq?e?U~Cv=KS}1f=Sv+m&rp%`EC`*>*!Lg#^zv&{t68uaHJ=^JE7^4z?cV z=8qGzX{$FW_&Izats!q!?@fjk!k_!WNQ=B( zpHfrIqbPBt;16_k<4hENr4#;1u4GztJ|1XoabK^bJ%rIF_a{2)af<#%M>nj>he`lC zYEadwcCDd2!*5Ab&V@GwmUQr>tnou@Y4Lzyb{?hM@g9g0Qq-&ItGogwF>V6TO@usD0VfmxFP99XeE;t) z&?Z?TA!%v9A{m<=-=IJZ0a(;n0W^@J&CLZ=^)g&ec>kQiz=0lwZE-?6N+YAb{)VgP zla6_BpRRA8I0n=rx!RG~h-{F#+MKkCjSrk`Cy!tJ1=s`Cw$eAG_lB8hReTQ!h1|=d z9?b(^E`OxX9IuxVHA1YFt{X8bUh2syxwV62FaT^aXZQ^}_^xsscpB^|cm9HXXhRg_ zkq<9|qH!k<5^RjX7im=ohZY-4ym7Zm4F2L4dXh)mcS;<=0c6QE}&&`x$(|-D`=aR~niQS=6zm z09EGc0&%Naz2W?^cKgIThyHu|^SA2jfBx_PQ$IqYV&P%PCqdhrDcuaHXVdY8!kjzlX@p{cQgAbakj(M+NS8i)*w-MrbqK?HZkuC0>=T*~4g-Vi@A(u+4I3vHUUcyT0&1i<9Kd~N%j>0MDN=rDeg1`QYCY+ zaXxX-HRyQ-6a3T=+%yeTtu#y!#jhUZV$*8+W4mvU*d&k+@QV-vsL22K%I|+yY5z(^ zO;Y}=(iQ_Mt=0cj+WQ~~0f|ynsL}hpGAa`HRjv#Z>ISiHm1|82R1&= z28$p?8o@xFgsl0~#`8V2mr9Jue#;#4U8KA*76i)~4(^LolkL~Ilg8nAUwO>@MdtlC zZ5i{JyzaC|Ff?RI(&q7&SEBWUS13^}MeIA17{GO$;of$101bHxDqfdy&xQ%FRL5BS zk?WLxcrQ$HCxn=qr=3@ca<9?4eN(E0L2d~8C&@$HLqE zrN0^unGRgPq#tb7*g4CE-^4)m2u%1RbT(T4bnR_Q+Ap7PoD~wXdyt2g zuCdoW-HhHam<~zep@&q5nH}gm`2t@7AF0DcuQl?`P%KTf-!RFV)&PKq*ic|q$L3a$ zy<%5qtiaLaiK;~yHDL0ii(V~R^Z&s^^m{Aoe^*-Xzpb?YHV@=qD(yd$+uvXLKbY>H zP!6~mu8sqk!pJd=lJv8P*s-xKl@qjm%F~OLmgxNy&jxC#M77wA7Yr1PM($#kKlp8z z{<@8hx=MHaK&$(?V8L6CF0j}d>s*-Z_1ydMOSdv8 ztgyu7g*e_Cef8nfETtn6gHrfvb!2!s6&e_HV0qnTyUF23m^c)cp+?FCK6yscgSNq? zieEm@KZYltr8S?l7EIWQP71Ga-53VJMYw?zt~ame;_HoskKSBDIyxI+SW;e7 z!Q^^0fR=jYf_E5Uxchj?QF0M8aef+M>C_|Qa$asVOr?v8vBV~bz(@z(RHk*(6aKQ* z40B2i|6j zNQ>;xp^jZ0M&16pbk|4~weh47oQmg%1EsvqUWX=bMqGH8jR|U{<+g$Vv4fd1Z+bGB zg69mdRyt?}&n(R)-t|Q5`zh?=n3edb2Pt9z#3J! z-eAvByUCqCxP{1RVIYYktVvGxi5stsP-7{K8-;T>O1|;GJ(bMK@5pBfNy_qw3J5%t zs)(dJgSrOnlR~i*?Yyo#yv5fd;5Wt~r(;@T6GqN6`#(o}gwt;A9ELXCyD=3+B(AhES^OscB_FD$qs3zkClIzv~^v2)+Cj z$!^rE49oKH@k>(SJ#5q*^Q#ajy5a!YtTpHp;Vjr~Vj3FkT#N8$>qg@&7O*vPw;u!O zcC()-<`(I9Qdx+A&QN>MJqlyRVtIxb{Uo8SGm%+>A}bdP%sg3cGmB%e(uLgx$iqbj z9U<`J_ERVU2l-ZvY}bdcsl!tZ_n*otdAk5P_S7g`Ay1@Cn-nkE9`ZMDFuz-ipL0TW zV1QD?{I@IW{~d_^we9skAlANW%bOcSH$F@O%;yDw2>WMvz^6s%nsIW@6K- zAs*8wRE%{8L=OQ@3e1x!AhtGlHZXe~9#5SAe2uBu`otqr;-|U6G+rLSwe)#o2p5MI z=L;6P7?0(H2p;vb+&B$j_m7Dc5+Oq6Q~}MBO68}M=X+vabqJF^uK@tViqn$4v7ABR zzC^f`+y^mPPLlxTFpHcWjl8y`KJo=LB|U(mA&;t@quL_`uI)iUiKZ)}-lfU;#IO^`;s7 zjbWo#W%h4#|Z#eWQ{`jQX4vUg!4^IboB)Y~+_$|2d2x8da zeTdP@=NgO|UwWUX%2f!Mm}cfQ*(>IUXgzweLzqJF5twHG2IMdKQtx0^QirhsV&t1V z5=}f1d5H--tdy{yQj((%Nuava4K;gWSRG+#ESAfKD>Os$S@}7|T>pvVbH5CVTHv`nmT1TPITA}IKg zVSLX8kzdRVdwgIM(^l)aH3-7vGUId|n&&u? zpq4XUI04%4#6Af=YglH2WLl&|6rrLu(jDnRsElSCWbsmAoF6U zZP2ih%*y;GXmS*b?@Tpb9G<|EVs)*C+TVC&Fer@U7s9230AKBHLEut*3XnEN+eEN1 zvsJ~fZo4^{OJ6?HowaNe9auw$pqLt|8<=SxP01cb^eT@$JWL>6QbO^B8BR>-nw67` zKev>@r_$VZiXTRNfX;oZ6JcL0&d|Q|_9pr?>B6%ntPg_Mbf&7+Mc@@~#t$jqfBoWsA*;nY${saqobXhq`uaz?imu4Qc>z4Ey#I#} zi}fE5>wl{oGJmQYGi+`+{UCppt2~6l)7Pa{5^XG;)iY>5FepCYF#iD%{2p;aGj`iP z+3mI4sj2g{0_KK2`>pR^!5kp6xXc`N>Bs48DqCCqGT^n~?HDKUZwnVOL^H?LrOYV% zmDtBaW(Cuv&<;$~!+AWJc)#@+RK(F}wP5w}$5|Hd3k>N97hssS8-I;EYYd6^jlpa` zEBT{h;r*^yU}$qmWh<2Wml0o6uE-5OHKy?)CZ@PEo1=IM6SJHO=(O)OQ*@N$ngTz0`& zK{6rJLD;6=5NCL6kUR@Jo(>K}!XSA8a0nTH$s09X?ZXhe1HZ+8I*TU7zIj6y%Gf;J zv@B@c5uNjJp$F9Iq+{mFUk5@y_}Nq`Y-yrd*WV=P!z@b6KGV#9n9qM`+<9-h`qJq7 zMb9kziW7SzyFHEO5S~HoAZgS(25KSt3{5};WEL!vkzAwSHF}y|&36ba^>JpB1A1;5 zHRk31ZDOk&16e2?=j&*ZPIi(A<88*UAGEu`Nct>^uqhHVrU~9)>ayq9$uotz3BP-8 zHc|DewLi>FF$aU1|5^I~@2b^b4$CCKTIrwjq$S!UTl7JCl^L*A^ybq51S=U>AmjSm zPk~Q)+#R?|CB^t6-;%>Db_0HoXgirG<-$HRg+3q)T1Q5wHUVPGN>IFLC+e7-N_+HN^+tw z)*F)%|NAZt|CgQT5NXT7j|>)e1IXbZ3RVC@&~O|JrNQ} zaL{+u1D&gdveiNu*PaiwELM?<&QkCQmm{0@0DxW=24PW5Wq|lFQe zoF@H~V)JK}ic0LY130^bzm=@h$O6l1Yf&U73t1)*)d?jk13{q?sfG(!l{>ky$?eu| ziayfq0&zGW0J#IpNqlzyga^oFm?H@!>xkQ}8@~I-Y^?qIIz6rb39mq_=d+s*k3-#= zQB0o0nyP;bb|)>S)UYANf?`h-dOG2UOfZvAFN1nnHY10)VuU~X8562~{9Um^NhemU zc^lp{567NV`a{pjfUaa@$b*Dyr=AdjJSq59kRr|c+)-Vm&=m>PnbNR78$4F^pwRHX z)c~M}AFS1V*8h;Q&>uw*^RYOkY8HbJo?Hb!{rNzyzY^r8nCAn|X}wT+nO=yKLc)v0 z1!a^$Uvr*m=xv25ysCf`7CBg(Qj}+%SZ?8sfkfJnzEbIkMG_A0@)2MD&N(JnST$>_ zKk~>_j;SWwo}xv5^ym^CW{WkP#h_GNDnu-mC{LRzO&6oPRe<*~BJJQ9l$ZWI7iQ?9 z*V7;U)VzwqhkGt)_B{3TP8TaK_)Dbi{_$vrKG1}U_#SXocC4w%(bG3y?-I#qU>Swz zArOwA!99QFY@+B;g`oC#POn364Yira+Ug5HsD*&nGNsT zr0p-QhYp=u&6A!am%qKjR~>%Ac>g1sPqH$G9}S>H!T#+~{QuXB`#+=tbxju(Rm^uX zN$v}l#v+_VbPH3ZrpH!c$|No)Wk7Y;X&dUtrFmarSMH7|07kqZJA+`j%63qxKgb z#8RFWBO}dZ2G|!E1-iPX3OZzr6QzLxIg?=!^w+zyC3r;QIJ1(l&;8+dQm{U-nKitb%&d=8NJ+RdC*(WbjAlQGXgxFu~V@H5k)+O^4 zQFMJyEF9#?VLr%o#&ZK{7T8=-k$^3kH&pc-O;d{?HR~%AHB5W^o|b0IJReg3gJCzM z^~*{yBi=dgf)SVYXlCxkq14#%WVu~x+D81og9c;dGcGQ!O5?2rWrsjxG`(+i0WQ{n zT=0rIq!HhxEAEBKB=RoQ;I3z?f2v0#6O1_{%AFDa zW)R-GodGr57P-}0z2yVYy&EH1yZrM5U`0(F*!#hPSzAU`X0%N?7@!7ElPDX@d|})8 z43@R|L^_YiJ6d}GR@s4dz05&QpoXf8vQsv_UEIvOUENOdU2h;=kKu)1=jjxtWJG+| zyTX|mwKh(b3PLalCycVZ#Uk|%X;2uYE=L#L&sj42#Pob-!@9m3$Yf8Ff<)3VYn!YG z9`)948eujP=PX05t)JpefzkJq#);@WZz%chcAa&oLX9}NODzP6s8dG{{6*Bz8@Kp>JBHciLx%s;L}m&4t1De^VD%`}9~E!6Rf{@83! z(J!Z=K6$}Zk$K-QSYKUEZ%U(gJ2JK?ZdKzLfKp~%YM|<2w=d8R4@cZw>UI>7DRT|# zbZ}Y`9Y&-neV2RgcARhbrCo+de@HHya1Tq-=gKo$mEg4Tv)6~u7}Dp;x3bCaVCdB! zaanmx)_ z>jQG*UD5qTT)_p5?)xirIKA^M9^JlSMroBWm{~43b#0tlC#9iHpIPk|iMB%nSZ9u} z?!vU?E7H6e;iGy|WbS?X2X&vFv`~9E6NVc%mYFg)#DPV=BC~m^be;yR%|e8Bs4&4I z`Qa^BYw^LhRo;@gj?b=DMk37lWn*h|JnH(QK+nNqsYL!Q%Ruwpb+nP^Hym8!MbeDY zR2ceEVEfu$92?;mNEWc}QDAc7^4B+8;!??cwt|)c{!1V)ITM1C)!6x3xEAM;3&^OD z%Dq_T_|%iYNW}eKD1t8rlV-N zi!B=74U6nr-d&5!n%+%|>f3}o^|kl>zeSC=oP;hgh)*w3(2x*$fR&nVmwaV(i^C;E*iTb0k<*`EZsqlCqeGyKGt?`Sj;MsxBv5&DE{4K5Iy2!8pO#ej24SIA0i z;~rW(L|)8u%Xb9bX%PH5^r;T%1q^-q(-B(_w|T>U8c3jv_Yfo7{#yb{P$5`n8HtyX z-%|CbTsF4Fi~I^;3ju%27XCW|%J%2WD*9{1uUJ>ZVp)l97tKC#4#u<>M$gw%u$-ib zOo-veRhR{k&{V3qPwlQoDL)=c!E2BkrwCW_};< zH*f>w>5kc`%+I%s0}SZmKzokIsj>Zq;Ten@eYR*{RAd+}w1bn%vHQw9HjE{b*&}I% zYhL^2#{2F{u(;mRua!_>pX>Lk&v0C|f3|D4*j}DC??sd|!0B@|zt`KT1E+P|u0#^_ z6A6<#hdWp8Xdj$I2QVE~ItFLg*LoDan_e*W|puB6`r37z(Q-l4tFozjSC)j&o_jZ z#}X4dC^ym*hMJ~GQP$z?S8+chEf~WAR$Y^<#;nXE>tY2iBnTWi5I?;d zdn=`W*5mo-CaDd6qhP{ix&tu?6Lp8E>`_eKj5rgeWNZQE#jJ&ytpE~UoUTUCAPUrU zFXw08S{*}#I31sJnEZ~y$o!ZgIVP|l1vx{#+wSz>4)Bm_N24B~7#3AvZc5H^EYNg2 zOp{C_Y6KuB8{{I7c?>ks7X<-3v{Q5W2|L^iQr*{Bx>r6Q8*l<+BTz!0zV;xxG5W^< z`G5An7J_(Fw_q1jtX8T=R&5WXiaKi*veIEqy8;8H;Hk&QEt4h-X9}kc+vbe~>CqEg zrjh}>>Ur!KOsWm~%XrTE=6JyJIdOD7@9d{6IWiRQ@UYiLLHCc*x?<|#{}{-qE-d|E z0&uO0gKJ8tF!>rtTIIS!COvhduvvpeuifk3O|RYb&0stSnUBNSfH0bpr|vWoVF`}1 zgOHXVS3j;czg%TuyuTkk@qcnLYKNz-ZF6jOY~$>o)U=x#oC&`}+Q1i1e1$Lyb^I`Z zg`OI&#YS+vV_=VQxTApK87dgo+=s&B9o;T*-?P;7BJ(#daHk9A$c~AN!XR>w8>Dl% zLuRLMI8i5ZPioPR*nYrV+Ka;cF`&KCIk|o%a%~5SLF_L730Ug8p@o{gT9fT0y2a-u z8r{j$C+jr6`7tOF2-o#l2@!)j+9xsA@g-T~!f!dgn1~j~tZ?g!aONtL?OL>*{Did= z3H*JxmHj94O*X-im0W!8TD%3tF3hYhw~fhe!MM`O&Q2ZLxoo!W!NfC{l5Nr1vPH7Q z7h33)l4a3p^YdkRs}DHQYzpp8MH?^XU>a2Wy3q1Vn z3imM=s>JV@*VLARbywKgpD^PKaZat{=v>dbE%`y#aYXx`B5g$i^46;mF)t(^WMR!w zvDR1eSLnDJ`6j18ZT;4Kcrw}ulVW8s3A9JLw<9%^^Us*~X zWPbFeE5W=C>3~pUti)`C7)_iWH&2{n-QLf2f;5yf?g+0iTgE#z8uhNoIKgly)6vsU zg(wu2EM1)+BOX^_(r;ERnu{Onh%4HwiyJuc%D=ql#2Gz|T5r0VYEds_dg@!7LsZaC zaCx9_CQQ9x;oz|8Qdd2kWQIRv7tz{?_uA>y7+|n(t`y`(h_mmKf$?sf_xt!Ql}~F* zHh=c=J1g%X0mI)V2E%v0gJRFwWj1?k^QXTijp9RqO}Dji zLGIoK#b_0gbCyfPI=s*F@8=K+sjMz$-1h?GH<0O#`6<~ud5or^2Q)DU0fwC9a-O=1Bo8Sf9&>NIn<<0##5gJ)|(wx*_cy*?Cgnc zl{z>Plo~=`C5kJ1~1!8v}a2>DTxAP;DbyZJ+5tK0Cfym!ki{ zwqOXxbqqyco1|AxXj8y>02h_fCN$=gvUT<7MU_MHEg*4A+L7e9#+EYB4u`@v((;5 zX@P!iG=ocd!wh!Yw|5>M7F;BXvAHD_a)H?}o>l0!X(tt~dUl5SrBs!<8lmu-%; zQsRQoqJ){biwsp8l6xHvE%ucmp3&`G(KiFAOK6h=`$L@XU3iew?fzk0OpXYYJcRP- z{#2?tB<3DdD<@u>X`+N&b7zz?X>#OXL356(Bi7U8X%dr#+hcW4=cRk{-2+OHQR+wVLQO z<)4(l%AiyI0&f#`>;kQz-)pzD`GqzreXA59r4~Q-T4)qu#aybjtUzlM$D^kF9qgBV zI=T_^%O6OJ%a%Fv0chOxkpGT4^Zh#{{WG!TuW$vDKmP&<)WQJws*9<~|7jssl>L1v z&|7IC5SMpPhwGFQNg+$*4r*a3D)Y*zOlA@=5eTM&=%LBq=)62 zL=tYskpmU(eFUD3p&u54e_kOkvG z4iFfro-UA3IE}-VT23D%8Hjg8uKy9?wV{~Tx&E3|-k%#m|6{C=oB=a{Avu~LUF4qq z@~yCMfXs@_snmMKh~rny=ioVDpoJ~8Qy~$gFJ<4_W-}8w73N@c=2onJi2!!#Rf5?^ zedqC4pS20y>O^ufX~dF+v)!UPmHU`ou&W+HG+vR?yl$B^&V=C#u$o70)F zG=%-!nKb0B*$j#)h^Ls;a=-s!NSvG&zEc2@Q}e&Ixsv^--_Fw1`OkK!N=^57JG7ZX zB7+%>fP#ivw~!C+UrbfCM)1EF`^M-zx31mBw(Z8Y@x(S8+qP}njcwbu)7WN{#z{}Q z-+jJu-ZA#K$N8IoSj@4)9UK_fVFb2Po9dRU3+eRnqK(`n2N;9tR)p<$2S7NSYJqK+ zJ>esu@gSe%)LBhrqd}5=O#$mqpEp#}BeeWsQr%u>!S#5*M671;BOwnX)p;c0F?y)> zb+8;VYb+&8$f8T~dy-O<^@7BdhRBN3k<$PzXNradOco2SlIzrPh|*f=%unM5#MV@^ zV09+;3X>rj)0JPeyef;ALR8mWDca(4jo7v)%_`OC&MQAqJ#2Mq`K?b(!Nv9?U^(KZ-xsA@zYa%@7Mt*7?3)I{ zUhk)}>wH$`r@zj*I1LdOqArUe0&%YkkXgSOlugpjW{I49iT3AvhI8CMXR^L{aRtLIe5?;l3_o z?!&=>@yzMzYwO+UWq=YgY)M3uejs2M+X)VciBD)KOq-xtnbj&7gqp<`;X*pt1F?; zVIfRGPW(4z`wX=z8$VY_V4s$*Nan_b>7H4-Nqdv}kZNopm@)O%kkOfwV^T$c>lIjx zuns^2itLVhSDhud1F`)LnwMG$qd(R4d$;iQ1OBnSgCB80l4}sKoV{~Gmf`@*qby<2 zBf~T`mOc7$r=5u%J``2dyh*@=_S*=yi^NrE8%Bx6Lw(4ZNDOj;?N74C;IaeTlk@&= z*kqCu?7YB2{0f|XV)LCTMYr7VNQyfB#+vL4-}1T&=e$|oeKO)=kGS;bWmp!Tb`UcV z9xaHfsy&EZ!)x4s)-0r4wj&+O2`8l$pfVti=tb)y(a*NrK;I^p;o-hBOX#oT>(RfY zH^$Y#vbVWpR$#&IpoiF-${lRLXKcTUIDaJ8+>wMA(->RG*`b?0&krIm&l!04ipEc< ziGBV(nCTsXR`b=t7IFLgvF@MQFy0D5rvGy;oQ3_jiuTX5p|j2Z2tu6|`egt31c3|* zmHCwtMu^(>`is_J(_^Bb1cYN>6;-!_wiH{vQ_<@%-0na;DGdKe0g1OuZ()8tn(Exl z-LA3yQo`X72bl5?ZV%*TvAS62sGTFV`X)~Hi;QU90s%^u-U~IzL6?d#OjnfH<3>!W zTKQFfM=sQdRHmCk^F?DCGTH|-x=UhGNC%agjQE<)-%-e%<~}(MEZ%F0NjYd=)ib$! za3_aG4Qg|lN-kKFZ>*PHnAtGyfM@hVXV#$jGhxVS?%+frCwwT+Q{3G+a zUvx_Z$`~hF0-0ui{Mmu&J;t{N z&!XQl%@Xr@dVU5{$-qU8=0yJW7z&VZ#o!U35ze_^6B$^u%No86c@JqT*l*t!TJ$OEDK5iYn^_X^p*MgS-t4DBSwk5`QV76r< zrN_q;rjXzXgF};Oh&m9%*D1i|0~H8v1oEJa1k8SCw%LfaLtNR{K?)(@^{M=6*aW!{ z+hn!&vw$5ECo~YV7fCdR$(}gOuvKpGU2y!>FAPG2hg2`846{6oKoM4HsR<-J8#o9E z$kCwPM6=!mZp>ZlHvfqn^2XS(y-EYPJ+a`YKBs|}1z*B)Q=>Izgo%ie6oJx0{H2$; zSxSOZ6d#ZxTl|6(fRdAatk^s@S}Tc(uEM<|AJRywUSigQiBx4FpTR#VW5Fo$9+*u| z$yVk}l#a>5vK)=kN=7LvtXRuJRc3-lt`N1=7H8`BI-?O9QR@>XpcO;~i(gK2ji0@>r>T3~QW%Kw3oJ%Y8x1lp z=ggFQfkr&noHa(qoLMxd8oz3iR92yRq}LusFeJf$QCgL+ocrE8MX(ys}OI&3E}D6 zk#qx}VTwH#=AqvO>!jEv%R;=Z4^q1+4@%@9+GXve`#xb`ce<*hj*7kuhh!`124gGb z264^wGbiW~YaW5rD=Ze&iGvyy(2;G!l|xK#*`}xlr7myNP!G7J(r`IN(V&CJA_J!u zQ|fhq7!eiFLWAx}dpJ?esy>pIA#^3@ZizHe*LU7$uZxbIafsJZonTk);MFA2o_))# zp)r&u$i-ZQ$1X2Y&Io5>b6rjwx%bK`7X{mCS?>41elST#(R_)^v$P7;z+J1KJ{_@0 zacqvSB7FF5k+Rr?8+>7BBJt>)1O736*Hy*Yj)RHqDx0TbMO78x3ho>&$(gXQ-V9sg z5`q89RSiNTGck!LQ93NeYhK?{F|k{8UHs!r%N42?z^qRqGJv!UZ~SG78K0X99WK3I ze?&z5oG-(yz_#}AJ2^VBW-Tz@U`-)C4F(sI0XxAhkqg{UdlmCSn1qY`4I8*IC(EDD zI(*;~E%4{zHiW07JVs#mp(TVN9dZNwPy{$2i8u9(98hvkG}s~Gsl^P(EZF3`CK+v? z!5u-7b!zsiPgx}!;!ap^!eeM);%WWBS)$b4q-LhISf+I-y80*@ENF=f8>%0ttFh;` z(cjxvqs5X0dBEsfB+-aMgoJ?ZX7oR8H8j=ZA#F;_h~33Cm6NKE{60$nuqMaO>>jZv z(vB_U4QKgci}Ld|}qJP#g`H)Kc!WFOrR}^&d_0>m)Jxxq?Zwv<6_1w zZGrQv9C?a2FfXug3n-f6S*ebZVUCW~=^Sc6D-S{FpD}9H;T#C&l)VX5Yw)09AJV~` zv29OaQ%}h+2kd8l+Y-MalCR{1eqxAis0zyq;*La%BTCY1TnZ!*tJXo(;&@7~?jMXS zuVK=Cr*%h{3hgMVy=Z_AAzb;8vHvJ5lO_=%D4dh3D^I?Sx1oPIezs>;sb6)k6azCE zvrvRP-%rfFiBp{Oi;$ijbNAIYtzkX}an9zQ&I+}wjcHgJOBnacD&HLqcqap3ybTlk!4|TvTHHOKS9mBO;?phL&V+zAJ zb<4NPopPpR@l8@$u>Lcj(>Kk-rm29R&Qaxw?522G+ugOtGOXTDIP0}jr~pzNrst4@QHj03Lc07KvsD zZn8m)$yyjxt3&zPO%EtRiJD1J5Ahsai-33q)@8fibr6eZh7Xx{c)7<##TvVv;CtRHcl7oCda)^;_da4#+5|?dz>%pN#AiRP(;}+1?D~k4D)-uog-5`b7t;xnGd8gJ_=!jF}!W+h_(f=W*3~@rPkry&7ZD;*lZ$>mR4GvjlL%-g9@EWfS zqWa`^cWP;xO6pY<3Wb7i2;Nc2PM=)1Z|*rF+xK?@^Vd?x?h=l(d_@UQp^Dy`Aai($ z^kpPiWWUX|_8>Fs>u-5Cfu10-t>_X!)U|EPrzD|R%3rK`MO922Zb2P+vR9f%d0@Ei zh;vMmU7qA-qRMZzCt4|I7s09l(UpGwWHP*fXkV0>x+o1zW5r^0zn(uA!^bShY5en2 z4F9V)E&e}U3`Jvm=g-uH|9n2#>OVaHbXnRNCJ97M@@w^~Ku8NtP{vB;wDVxJg^@B) z=o`g}dD%>3Ng4uY)t@}WvkG5-geRfbvB&3&V}iruHrKCHObdlY2+WQ%yvME&j`kf*(8@Bq zEu!sDW$+_nRG_g~F3nqQHenVH zz0bdbWo9AkaDI$QD$TpUom;p0VRg*HE>gHV07B6$gLxK?^O{-S6$`x$>5pEXk{? zVYxu~VH(LzB`G2fgZp9SzzfBiH(Q)i$Jn5U=QXSibx@PBuUr0tD+&qc&soBVjM zWJ0!!xwBg6DJ3SwdI`uYr*$3pOr@`2q$)iDRdF$1pQBYj$=xZ1?Q|tmQ=6$W)I`aNmxl!%wJM^Ud?mg`M)p-ms zN05D9;gEeT5JL`({AP#dC6$$w6|2qppJ>2~aBO7Tcc+E9C@XVt3LOze93~(HM$Uop z2+Q(cLK^aL0Kc+#tyytwlmKHdoa|Lzl<|hq$edH|II{O?rwxo&SaAWVtlCmb6kUr{ zPy|)E+*!T)xbZjUz`473pt5_)zzd_u9;4DtJc8?%qy!-(8~d^JrB*p!h*D2X{xvtw zz^U>2CqITOjm1N;&Lg8!O%cEb&Y5hkyrH&9Z)m%vc4%~VlCDqp>NHMb;p}1vK2;z? zL+7~>y>Aq|C7?dTAoxpIANZ>i-1pk$Ji)ril@`r9V53>Q=g-KW9EGGz04 z9%!VnB;5hsL zZknYh!Y;b!k*r|pIvqjEP_oqXogabck(wzbblJt^MZpKj8L;;lx#dY1912$PV4v$? zU22HIODuC{`Nb!6Kop+=8YK-&_(0JPwx}PQtxBmcMeQ*wb)x{ZF1GQG1I?+Hu|*!F z05T}ZysgF~%aKL?fjM<&z%P|%J!X}nApt-XtNQ#4E@y$ ztQogA#j^JLF7MLtx9lpC2Z49g>Kpc6V%b)XSX{}=a^2nTfnnq1<3r^xWjC_A=Shim z|H`HW))%IG5cdQjW)bd%lisM>Lzc^-Pv^i+;S5c4XZS$U6~Qt5{t03zF-GYWfLHbq zb&zgzx)R-Ns_9)=t@~n&d(>7Id9$p!&Per|S*He4beceQl&CaeYCfZAJbnqJ=cq;m zSW9IvcMkE0E6hm6E-TRXDdV0>kB#OH_Sf=nJD!KNy2K(*T(7J?<=<$nJ2fA8D$DMK zRGPjOuM+iBOe^jDX8V3O3xWMWTfAn5f$q9Uyjj7Z_P`zD+ZCfRXXa#hTj!?Ap_cV1 zdrWds73?7ogtqM$VHoog1ke_oMmUiEQR&@Z`QD`PW{A>Iy%$|h;V}LtO7yLu<~(V~ z?+zj;C9nzocSCqe%MIJ*buV|oZ*3luQm1$H*Sg<-VawwUxHQ+F{TIoA0frZ z(V_{oW0I#3UDj6~S6$b9U2nM<)n86}Gl86Tg90AGcKX7}ZuWr~c_>0cf!#1;Dco3k zr=S>lzJ`np+0Dn21FHu9JKEdl^`$5HZt9S0up93zj6w`lo{Ak#DC(EVF$vmaOa{f5&5Y#b zD!de9P2|N7WyzM81Wm505;EmjED5cakdk{2*9nLiGqEQ1VYY-iAQJ0GadjrEs1g=# z!&BtX{?&H;DpAvdhs{b$)S|N_l}3_>+*=zAd?ZN5x(hy%OQkm6i(PGJom)SBq;anB zis6L~i*n#9Ssu-1lV0$O8HL3u-OiJNnX_EW6Ra#~4T!m#?9noX=;12QN)B_e*{8R0 z*ieiK6}5p0IA=BcwUsi&PF&730XQ+keshbHpk>X@lG5Q^5+0+ETEPCFKzwttgbWq6sD|34y z&sJ*+QXO21adIt-&1pte2P|@Xf6+YVSs840S*1js0~0oN{PTV_3lzJu{1CTrW`?9J zJw+^>anyuFwOJ;8Cj1^TWu5^#j#7Hz*L=i@auTuk&b`y)c$3V%RwAk1n_rx*XuuSC z9m*~#9wq4Hk4|^_oy7W~Q&DIu68(C?RLZL!`l%6@Cx+2zR!$CT%54!g_oXLj1FH8F zZaPD-CS;AeQg{`Z!#ZfMwYtF}UD6|F%0nWPWp33=J%2Fox^5cyFbCXa;lM^U3wHp%usYqy_%02S_jodv` zDxLlrvPzdp$jwxy*dnm)PoK|51!lw03n<<}Som}sic}10wHp|;p1~T-%$5oxRZssy z{`AF>CerJ+$-_@Dzu(ak`b`INp+ND0)C)x71Vs8?C zxhiiWCp{58R8N@^J^KVyAK~n%Z&Llkzu9*&2s=IR!|;!rkZ6n?b0s$Ru9|vy67#!Y zYg_AuBT#nj>@#T=*u80dX(fRAo*zwAwdBruE87~YG+bW%^yN^I4n1JPy*fkgm4$>F zaRtZXGmG}}8>TLh5?Q}Vt~P{8Hg{zNSBL2&e+PnI5a#dpjG`sp*rv)*hY5$^Wbm~ze3 zR8(?rDkOz)HWHqlU@jW*u^bn`~m3RVu z{eg*VyiR4%)%kE8g6N~~bWgXLYB3@Q5-PLxPLKJq>+V6Tb(_n!2W~0rhn~9wj#?e0nBwn_lU1;F6N}X-STUMGi0ekzoy5&fm3p zZtZkjpnTd}jR=J>f#U;OvHL+96~wvmw6DXo|VU>cN08gjNVmqAkxC5<7*C8j6@eu1!Xt>V@25EiqX|ixsIvB_5#5O;}jasJ5 z7MwC)Y!t~j7>K8CPyu(@ce{hLkA|AgF49IKvW>RI4l!A%#cv}u zwhPBedNwT2Cpg+VD{BvD03QUF?zi@h*`i$w#XaL0fVVIO@J?Q?r!BNF$GRdNcyGug zo&yQFM$CC-a5`k@>JSIj$Pb^xhWWLox-$;O;r0reTu_crRSzjoY1ffmN3Fzp6e(S36eVC5O6mjXP%R?m?0W7Qino*^i8Ii=&ueMgPogX%4st>&vX?8W5P@r54w5 zxkxW8asef^Hmn#el8q))dZZpm*C#0?a6e(UIPI>g*olwv?#yBYh(ff~;qDB7AtoN{ zv+OGf-(b==k4V@f7b;Lax=bk;}Aj9Rj z>SdUEK@)IOU+}RyNNxe*Cq$L{_)AgmAGT#h@@f8I|F5cse+<+8F9__j|I)Yqd&M7} zplkV=D>F1hnrW3;-7NEaRaxeaFrjcr#7h}e;T6>)k8Ee(2yYo`eba?W>t0L4jFRFc zGk`#nr-4%SV*bV9czVOr_jr}|e z)pTpDACq1i$)fnuX4A3B-U@EIR}bSGHTC*E7cyKIJ-aH{o*Fj9BF0==)QmTyP~ecY zeeB(++hd7F>Cvauc2l|Lr`!Y{BO zn9hoqYKazLV#By~bQOlDQjPQO7*=(C!|tnh1w&;!qwv1G%@eq?%>}qD6qsVJMuE-e zH2?`n#nKvV3fIyrVjj|u51?Qor;nJ#DrL~!TU6#dIJ0Q_ptQ*S2wRLC7y`zW#-Z~A zrdFrLdicQ8p~UM}a;ka<{{!RXD1Yy+`-E|*{}smhN1n&utF!<1x-C+bal{cp^d_O! z^kj87NFyah?*rW(WfEq>4*U{n{C?LW3)OQ(@Q*!cp%(F^ne zqVm0v;|AnO0hPBP`2ad$9~xpxdZyFoz5Uqm*CW602bdlbEuKUto8VM=4CS|lTdwMJ zHxS!1iXB>W^g}Zlw)1(K(^mhM!_rDakn7P;r6oJ-#(eDfS@(}BoM}>vj0ec)*mW{)w zF9Z%=LD(ZBtpm4Ee1{D3D|!wrgd8q&i?t*A`nmhei=0-ASSAFn(pX2av>E!E=wzli zvdk;|uF3`RO~&dT2?-2Q#s?qSc!JW-?9xh4J|3KM+~0Wi&eW3jT=af$E;vs%A48Fk zV)?JvPks}0Mq4qTClZHusKAdaL?2KNOKmwSHeUZJLul;ec@!fiWv3grVP*n2T2SgD zR*7Xj91of^L(LkF#wlSM+WKe1Sx0UR?PHBkvq>+ITyUyvwh# zB(zOWh|K92Q?UFU_hJ8y=lJmue+h`>$gN0e<64c(U*7tPlO0aZ0Pje5%39m_aE%uh znYoe(vc#Xob7r_m?zboYoPcFy4WzSQ7ySfbMg)2n?DfPPQr1xny%=s8n$^Qx(?n!H zZeZyUC7F^KiZQ=7^pN~SoD1cc6SV5huUMjw?^TO9mBzqh8k=h4C(T4lUpcwvF=P81Jn*IOpKH+cNSlL_-<@24WnULmg2GbNkl9!LOVG8I# zG}Nq;DY&`$!8O(U##STMQj<`>7D`JY+Vm zBkKuZ@q(lbb0!$BWhLU*$IezPf*8U?uUS5vFfL((0oi(e1_2g7oPql_KPNqaew#+5 zvy>6C*NU#crORzS%}aO3v9!F+piHikUfygk)`T|`t#2cqcN7{-vj+vZY`=Y@H!rhj zR|W7cKK+ira9U$Cl_2yd9he{WDXhkKZ?5Bha0*T2KOA62nn7{%PlavhTHRirSqd4D z$4vf(lzuX7ri-@q)p&x5GD|vk=V_9O|5;lVzf${JQbt|X=Q`ABYmo?4pG z&o1jWFXB-}W-A&Z{VrQLz3ae!3R-<7IM!C5HhntlTGhS|qEIwVmJ$W3FX2PYnN4o_ ztO{CtIq;R?egsvE)k7H{q;kti>MvE!jqjW?6_9WAXL7!OAy_rQg`)1CWRt9al_mZ2 z^26EYk0|LsmpC|HK?+z9apbqbcAHOR z1q&D`G_KTJVE$<2_rXmjS1>nR>bvqN=wooDib`~-Puo>q>Rcj7oKk(1r@x#oIEnIp zlYcfUQ2)yD`S(TS|M;x_wTSf23`*mGm+&txN>W_=5}x=;rzC`r4p64N%!_HYb4W;T z7_B4m6?m=DgwiYv{qXxJEpKo+NtHr{-^$K*zdO48+oKLwug4FJCsj&D1b>~Xxwb%0 zO)xvpO`o&Lhct`m5zD0`#}?-}o?2)i_FL+!KjTBl2IJOHi5k>R91%f`u>88Ix?B;a zwJlG0Kd8&62;kFNZl-riqse|e&Qi%QZv&0a7bp64<67f0KW9SQR*$nJh5o?(Y*Hc_ zR&Q2Es<_z@42XPY_|n1xqR^AlKF3W08c?yt+d@Y(Q1@*;%sVuiMJzw{Ckzonzxi30 z@>jqw+a`jq>=A0e!3a)Ag4DRLA-s^GXW&p-!nRSHEra9bC`BbRCew>#S-VCFZcvD* zQ*7wE$u&xw@LYG?{OvB;d2rnUVC^AaHL;=PSt-;P?sbn&`KOer z>RY~CE}ETwsv)97T7;LSxJ=FI&!;RIx}2kBkyWHe75BzuoguZ}bK5xpYetoMzx8YW!=G1wWIlN`Mt(SyhrmFW00`Lx-4#TS6iReBgs&g}IEVJiAZ9At zrL?Z8j;<*m-}V-elfAq!x+gfx#d+yTDHA6370dgmbMTD(rRT?Dj#xG9rR^mC;87Hj zFCA33o>`+}!5RR(>R1L8P)4t9k3%ck-*Ze78t1T$<7hhST{aC>8qAEv;Mdowipm=b z;ZIn2J43QZge`PN!=rJwL5p(1eey0tA-8Ksa*me8Nr`sTx8Ql3{(Cf%y%J=KzHwV z&tN;yV$h+`Ai;bO*t5fRmQUd3T*b8dbl$wG3>4c#RzXW8-Wy+*7n3#f2<6ZcEn~_Nc z0=j~2f|2V2PCub?FrB4sfliMYrp?8I#+W;4R?}A-AABT8yi;11rM?Fbm$^i5VsWUJ zfmIO#PdG0S#s~YGzlQsO=@QkW&tq`@JgonnvqAP>2W4*ac~Ca`R`fE)PG+`7O12LF zb8yMZ)^ZE-s6JQ27Zk=({`=a(@-2+%AgE3#xHteO9D)$iVT9r+CP_)D_87>X-rT^1 zFOmjth2i|;Zfh|LI^b)CU5`0;8^=7y8w;P-Q!UqD_9*Pa?l4C{W7(uL8pwHa3?f1a z7N(3mmw`i$d62~&-1-7XSVFPz11@t8ZxR3z{__3#<&qwnNYrR${k_IV4P`BD#vyt( zN*hRqDgadjb8AbjMG?RP3hb0fTV;f(N-E}X4Jes)1#$Ud8eJ8&;#k-rnIS++Y7)`E zCT@JjR~&8OE_l*F-=9c6B`Qi;QCM5E!M@Ol=0LCRo~-{G2ZfoXsEIXUF(px@Ony@{ z`QF%Gg-D99Bw5&4WYO4Bo7sb6V+$YfT0{ZsRh9#-Vsx&ou-&v9cwBH{lPh}GiAjXT zSb^A+lxyC@H>4$@ksOT$?n==q^eK^JI>jD0_uy>SAXAf8g_BSdDQYe$J#Az|h?caX zKbaMZfzx;{fYY8|fyAl!nQyJv!7)HB)R?;+GQDq7dPD2UnYt54qE_xk&boOO_0(?-lE!*!O zj73h`8%3isX!B!;aGdOGLMUqNF+YlP@D$4CHQ++~W2$E>j)s&t)xO*v<8%TU9Ot^c zr_?hvGvOY27&&_ZECk*q;os~32KNHihE zt1CsXL;@-7Q8G1t)K1!X$RX)SfK3NUx2iTuNBIG~qO#g|fJpOBc z_}{s2|MLZYdi2}6>Kj-YJCZ6neaf9k|8~}mR{UdCiJE1dwdxG#KMzjPCAh$b0Y!cY zlt-wX)JkEIp=_^As!X}ouDUl~7k}B?a!)U$EAW}d0SP4Jg|Jr{egc9tCCSa_=`wwR zb?x)<>ocl~DT2yqL!~ar6F9{*zCXfG1>~d}$BM5xgU$Xo++tB_DI2a}bDwsz)5Z%V1c^qlGV?iH37tG=l zxM~Z!eb){GTQ7Z`!t`GTdg~u?cZs|yZ{_VM4c$`CLkD@ol36I+2n_)M^i1q#P%DxW zqFCI4BV~9jw&I;J3I?7WCw4VAdbPpBs<%|(?lv+XdzTVr1NE%?NL}rO@qk8;RT~w! zg-Ui^C+?6Rr&mq61eo+%m+v-BD$JwrNvXt3nlkw{+WLkq!ZRu%n9M9_lk+5slxzM$ zL8GV<6BoRz9cheWX~f_+VAy%+Bhb~mUkC3pN2=`K8NGw|viBe+vq&^B72-HTy@1S} z{0~lQbigJ0YZiytx!$>Lb|BDin3reX<}lAA&43~cv%cdg_+~PFmkkbNnIc<@hpJgo zK1&>~4tTXM`uSHBM8>=l_aA57=6^-${+AbB+1SnLzd^ca)mKkkV>F*0BX;gh=>(U= z#=`r0pbQ%}b2WTW(&#!OYdxtogM?B)I?+fo%EB-98^TSDb??cVyL z?`|u{w-CSKe?$M`Wo}3%&73_tb>-dmw7KSN`2BLdehu7;^7~?tsgiUnwn1VeT8!OZ zMLHi;YZ@18oXLv|3L~e|OG5?OjRp!Y)`Ell&b_UTuwvx!g*o8nhpZQTZt5-!c1~Oe zy~R^P2)$)1MTzd|pxQ0#r-o*S5>5_gc!iZGagfdn5Ml~;&e``;9AazkrjtT%)>^nL z0=K<6GP2yX}PbVmgf-i^VxH#(SaJ z%zxZbtiUDUZ$Hue1{J){%pB8^#I3cGG3nWeka2F)Sg1gK-w_POsYC-`MpHULUF6)h^A^Ol z@^>4(hptw6pdEqN-fbL-PL8MG$t27ugUahA^U5&A&)dU0J9}tfTC>hi}SY69> zM=TzuT8*IjveBl0l!bj=XBs$1ENY-xhmbK8JTOQn@pm{&Z;@24j-hY@BaX*D;x)kz zHV$SxhY#*XY~Irv`j%4;yB*(X8R>jFC4(#kyG0fG`wm)5E#X|KO|HgJMiy-9=HHnDNTD#%U>{%C;IX+cogvE-+3(b(U`pc`Elo(vo)3@E&)zKy{Yv!T9HH^(W6b{3-Fk zq{(iS@YHUZ@kS2EW^SP!yERYW(JC;F+5j}TQWKH7C+dD>`>9XKJg-eL+dC~#oyGgk zoq7ANt)f!LsWL0lxy+qAW0{4jT;tc6hKQTv0zcNG*^3Jz&Y?Ps_Co}^qFqrRHk9d5 zS2g*=<>Czol%GB8E)keIslh*xY;X3S0J!s5c)rLe`{sFk8Y2B=`q-?z>A9SYo^CdF zcJk5(MXANZj2Ww}8L}&*G=fi`%!V5bJl~;hr#HOc@~iLo-ZJ`bhWx8#*!3N3HeS$| zvcbhIra`vr%crVQG!yR4swt=@`VY+%<|Zo?sWP5|FQ6jr%Xm{YJtRq{!d+u7$T+(&R5{&0r8k%eR$k@<`YSc>F5J|-P@5_R&`{9733h%50@CSDn_j!5Vv6jmd+TSRTIi&-n0_G;hz^jEMhXRt1g zpaK5?bDr9-&?|`RojnmUC8}cjad+0|!n=Y8AiX4RrDQw1s(yg1$7&FczyhG1I%k^6 z7GT_6p7y+8>z|upUmX2`3F-W5mh!j)20DN-c~Qgk41Ur^;OFj^6*`1T$f}y7mbiZB zcfivwyQ1=_@2FUvY>S{(DO5$-(IFL2<2}$DSj}s%14uve{2DxHz&KNSv|H)xpq6&O zo}0rtQEHVov0suKOO3Z~)}v+(WmmD1+Ei@Wr+Q$EgsVhc*TIOCxw%)Rih&!n(F-+O zqp!%0(9P2}yuA?U1Q%rYbkVU=1BKiwuKBp$-m~`Hu}*;k@=k(uS?It=pZEfENcgTv z2!S!y8##GXAsr)Z5WuHH>L|zbYQI{!uJk0{dq(~1LBVva z7$?j%UtdR>mu^9pds5uVwPi#f#ibmagWfmW3J3vEH#<)>*Gkz<*z+9<4Hp^)4m{xH z=yi;=>VRaygfO6B547K*+!DXY46fUv_GLCC;1tvxZ5~3{J{y{WxeW7V#f}={obiYs zBr@)Jw!_%b-%9l_E&JC9svVnxit% zT?5-w&pvb`5>OI#wc=M270R5tf*&t^+7Vp2R5qI0x1eS0w4Gb4OOa6>X?{xmCnSv! z#!K^08t=!y^8WvSyRyH*q-Z5=+XV(xULF1fV=TNtILy~C%TUb{g@<5y#4L-(rTLZ$ zO4i8q$*CIF#fyTn&^^JvjBY?OUY&OZh%3bb5LBvCF*UI>ypKAsMbwWzYRF3h@)n~C&7wod zg`A%yq(=>yq1?R&RSF*s8FJzBlkyOacnYf4;}lChCKr?uVMAW0gnxB4Ac_1UPKP3M zzi@`2axVd!um8lVEnyvPm60P@mi#}9?5M3LHm#yV2r&+62`wLs`R@Wi6d_ibJg#ti z$h>ZNXogf6{Q!qVxTjgtH`F>C{edC84QG3VOjs?YrJ-6*uoDw3fG<_jaer)odS7ph+cXU)1!oZr8%bD!|=-^Z?Jr*?UGZkGg*wli3!+T{n zTZ*{r$Ys34Ulu!*Lf^sM(bfjw zVE#Ewo9q7vhe%eO_|wMoCZ?jSxX1CimB5ja$_P!q5+M>tSe7XYV|>3LA1mZ=xY%1& z`Rz}?1EWqN7_LsZ`col{+htMQx?PXi@v6&lhLhR#T#Jh(d9ppEdimqnNVUL0M zVpKN1HJ&R4=1=VjB-;Q9UAJquZS?weJiCq&Ua8?bqkdmo87xih74F#)AjZ>_eVCcu z=I-M#L%tjCtAn_V;_8K48Y+z}<-~zeWah$J%HZ^|+VK*l-RF+xG{}+MD<8<#Dfy~w zb{{?x&iTEkxLVBlE^)34)$MkFcEn<`_Y^vorGy~tpRE;nVA=q5&hRTnRCLV{oY^i` z>|qcv1XiAYLyroP%|`7jOSgcM`iH2zXMThMc?}1zA8l?jI%=GIE`$$au;(nDR~ni3 z)0X$mo6HXsT}-eD_eP%LqJ|VSh3mVhn2paI2bgB#Wf}TGzQTACIU+5Tmc9S@QT`n)C{dV)j`NR`1;EU0i?*oX7Gqf}sN`cU1}76kjj9l#pe6XoxW>CQ6Xz*$TW=5+&Qjng6B$o3+?f@hdQg zMu(>*m5;|0Sy&!4Qp(d;8!%S?s^gxC_HK#)2J}I}sD;3a*l%Q#@ci-o25c97hg-rq z4T;qgxt7K^sTwqgXN!`m^PFt=JP(!}qBcL?ezcC5?~#nD**~OoxZPRdz1rG!9Abya z&CQ-Z{Q-)+Nh~odgz`H4q=3|+{IhFO8k3(c(Cl`Oiv3PUt~J7ZzkGUm+A1rcW|KeJ z{UJ7Y1ud0}6cPNyGEU=*Q3$-NTdSoIJf>ri;Q)G`+TOumC?T{!?C$jEe-Hbw;Mf21 z=^8kjTNxQUIMNH+*x3G=y^Vza^OwIr-Ov22PZNue5K{Oc!8WS3^1LPF1iiB&3_){7 zu%QIK#4wR6E@^NkhV(qgx#r)6-dCF5Lrt1>mlo%V2=6fO^wvGhCx-G}TJ+l1t92s?mo##EnHVthvo`K7jyW$7yHdANi zzE;Xr>yHfWxhX3~w+ia{xjbs{{Mb8P^=`hsn+>6(eVsFl2cXmgBKkllSYEPV{b)iYD zEMrETKB&RIt7^SnNm_np$eZ_-?ag>$7C9p=B&No_Ink{@fo$XQC|`n!LO6e}!;py$ z)eBBcSywul9ZL!sJVV92Oqi7N$N*v8q22i88&~UN#vSE);3|+~44}sR@`U>9{~_%iyerYOZQ-iew(V5xq+(kY+qPA4vSZt} zU9s(?;)-pv-ae=M_PBj-_v!b&Z;ZYFfW3bUbFDSkT+@6VRit_I_WcgM&{>(NUtd;> zVL}-_+DtvjwPiMW7|YHD=*7T%ZUn$x>x=l)V82@6pb~B%lm4jV30TlD7?h^pHL@2U z#n~F)+~DV9OlsmU7->Dv6Azr zcOlAe%Nl@UU$Nol6OqJRND9MlBl|Kxss%2B0+f_P&h}sol>p zpff393IwZ1WF2xobcLL^KhLJflZi@v0xoY*|K+7&)_w!oW`Vx%O04xz9`GKUb4wu^z9J`9=c8YJW)+o=Ec) z6aXhS`>h;#i68|=1Q)jK9fz449nbf#Yx18?-OC00N#QN%D%59@hIi2{X-;tB%}dS8 z)m5*9h}#Ufl7VfEn#D)#M0&R#>>K>??T%i>m^`LU8o0M&6L7 zXxfk$0|9~d{U>eCkg7+JaRU48HjQojMa-0)xsW?$99~ z;TCwa($G3r9}sq_WxbG)6Li}<*^oXNWZ~oTxtL*0OU#l44xEg{n$D-0!{M%d?rZ>t zaJv;eem^1@@O)0W@#I#}zD$3{QL6eO>qELfvZu?=c=a0mi>V4LhKRdGj@O5|*#5I6 zl(lNt_^iL`UmzE%V_}{*KqR*bFZ)PzObyzqsd_ISf+Vl@Mu*-u{jRw(pX;UCW z(y)$=ilb*wHw*I-d*IorNk3GOTd^1+%X2gqT&FgqS4az@6Ez`~IHuo$5KI|zW>%uO zq$xl8Peic^X zA{jvy_0LyQ2zI;y^CXmJnWF)I@){)+l72wuJzo83!PS)UI{H8aHS?DUivB-Xu$+sv z^*@Bj4bLH-#dW8tMZ$V1TU<`N z2i|osgZ4g`y$sO`D(c{yKtYhLjR~tCeey}0Kf!e~BdU{CN*8O?oW5{o&oe(Epl9WP zEN1H3YR~Z(K%o-5Oe6=cSf4O zad6N{-OR0lAg}l@$H4;s;TZ+3twl^s4P30973`dxEDSBIEu4X(W&iZDS>J5ba7EF4 zDOQXVOkn*N^y%5Ik=c3q z_g{8bginMEJQ?KB(8Xnh%x(u4PC0E(m+L&wyO(-?kXHl+L#G~Texq&sIy)PwPWAB+LQ9tO5@Ygn^*5IF*W}$J zlZ_djES{E=EzB_TI&xsw@;PskHhp}w>B)1_`5shvVYKg;#(bmpy z`xAb^wJ4h?MFe%vip3pY!}Wb@d(Z7d?1US63gKlq@+v-9vR?hNfoxi**CLT~yRiHQ zk5!B^TJ3gpVG8@hfdA3*!AbTt^;@@scZnd=EegOoBe>NL4P@MQJJlC@_=K95365*h z)4G-nH7oD6X1oZkGF@kzwq4?N|DuSUKEBWp4^vgU2N&-0t}y~Y2b7}=k5Q#F&=?d~ zSE>6R8EL0#PBmUy`tx&2U4F&`II$dp{G22Q9W*} zz1R{(JNBMc$gIHB@87ng_FWnlxTORVH}GaAQ;1Xu(l`94cab`xEpR)M%j4EatwenR0Z zRD?sp%w3vw$4%i|a?mDD)Yl}g5C}Z(!bh_NV)en=aTc@n3-{&qRXAX5Qt zmjV|5@x>2WJax}?j{{7)cfm*^I9`NAg(_Gg2)2k3x=kToQUpdGStr9ABAA%}9?l0Y z#y*^arr!zdu&CAfo@r__bSDoJYFJpD|KpaDhvX&q@IiEhHF}o@-!T|(JV6WGu}tD3 zG4>|9PU2#Zx|CWd7z(|gtaxNpqYtPejPKpyFk&+?lY=Mr6I*CP5vrO^h`6hB>&Z3C znK1Sn$z*8jcCcgiw*{@Yg~^w)2)u&sJ1|i3DEm;pcW3E7k}k-2bA_x2rg}o_mK?tM zY|Sh?esMF@t^8pu8pXNk7F2CJ;xKloQUh0RMj3=3OSb%e#h?qOKCh}6rKBT`D~QIoJYcP}{1lhED&JIGd_6E$Z~qt0EnckqcwF_ z`H@Mty6bTsOmil+5xZPtB+Rt0oTSJvpR-f;zMg~DxRo1`>eynF*f#Aj{XV7Om33NE zqg>sZY2`YLUItu!VhIYa6M=@QCD;bnMRC2T>jKi_lo*}>GhM&upm7sGjdud{xUr|0 z`X=RNY{OeXMSXWJoDATbD^GvzdOIe z+%<7#n`qGjK9e*nLigaEoV7X8F^(m?mfC~S*>N!5!zby5`R2Yjyu!?_p85cg^jYx0#m@&i0dNKL5$;sY?%QZwKZ(`@if-{`0A? zfAl2(0lfe00I~j4dW%vTmHLAil{FBFy=)%&>WUNzi25aaoqHr05Fp)SM9D50T1R;| z(fXE@4Fco)osfLVd5DcrUNS-TSlQT1aOMzgxWta%iuTF_dPEA=2Z|-K8XrSlLUouO z+pcay>LV}u67k&%9FUbKLL?@PRebvv>b{Om%qqnx-1eQxsz}0pJwKe|kNzHL(?$RQ z_{@)g32XkZo|)yJpIKQ)9=M*xH=le_Pghl>d8MV&NO&}G6bKz8g8QnEBUBiFw5Bk` z+`2*D(2>tS`1==VmqdOzu(C4w(a@n{j%$=K)dpM_X*0W!@q7OOUOphL zNXFlDSnI=wbgMJm&JKQs8+6&PFb^cwvpZu%TkXv@rgBSS;Mm=+Uf`=cRW!}Ry$fz3 zZDf9>(+GP|9ho_S@a#b}MA;dODSQYXb~D;Pm&CtGsiFS;C?3tDYh7P6LYU|X_ni|@zuil;PuY^0mLE?!`Fv> zgF+DgMg_`u8@4M^G^_k8XseK02DfOKmPYS6(wGH@vC&OtIyto>z07l>C7%_=(UED{ zi@dzn)ZpD6m}+L;ROY*d$>WI}iq$fADr)auo10w-(eu|pf7o5J)q*T8(B^RdvMTydb^YJd zjfsf=IJsX=OU%8|K3gpTo?w=l8=yhEKRh9`^RjqUevj+)F^hKvyWF z`_Zqsz+pI8#fg|`tn_!#kMK|?wOxCj1B;jVUi^`|9gS-6{+sUWN&N_=@(iU>>xi9m zuJ2O>!X`Y}iY^5VNk2#V5rQ}O`%`(QpDr6fepYrnxXRP6_#ai|NJ%lC8C?Y!O{D?edO08ksj$ zu}gh4Ss~}Dy$7y=sEP6lT+Nd3TBab<3)W5{BUvbItKm-hB=hOz%{ww9%)Wsj?d{Soc5vS^tV zVbd0e8n`}GO*if!tTaz$QMwDu{u5z9PHDKRUrciicQ2_xdRM_5HV9*^h(1M#N&yPZ ziJI6SdrcUk4^-IH{a2iLwuhm+Da4rD1Ichb6*g~y)5rU?-^c0W?~af8ZogZePmtHr z@Sjt0lv34da}AJ_0R-TfY(eDwkuen`<35tXRe9e{&{J5^6qOW{-v9tJ744vF(!nY&^hK@WsbhEC<+aq-)qEc8q~mu-(8*NtvF_!nPC z46%smZPsGG$mzD4n~sZv_YKpn(GX%nk#$T%fkDD@DeE7$nq(v-6~UT1thUB-?C%RL zjpQ%^lD!{NdvEeZ0}*2|0kA%pv?>c`D<%!*xX)d^eRIr3(%@dT=5c1&jTmeI(cQ7D zB0$_>C+VmcZ#oSoz1Q}UD`AUMVZm6u1`Rgs!BQD)iSK|-))5Ag!!G6B`K7!cKwnJYH)t{wi3&m|7&2jYe0CQ2>hII;ic_$s?0j-;J zL4#YV0;LLhwsVK>t>!pf&$Z)sq-j^TGe^N(wyUuFqD!^uAUOpLn3En>gMRBB`pG4* zwtMy2(Wm_$PNR+EAftN^Gs8perv?&C@6yC2U3V;(D^MD)Q`!b`Fws{U6+ezL(2mNL z{o5h>$S;R3=vcIBEv#O~>bWfFm@cNgnmA=^MJ3Tu(swgR1w_{_82z;pVH<6Oc|8cu zv@972rSVfzGwLy{3nmml+0W;9E5!79~A6DmJY--LIvT9alqj)Wq9;Yf6Xf zJ!;0A%7lti*~ICvhe6l8?!K$!79hqP9S!e-X;z7W>n&chd1ozK1#&6Y-H4P#+x>F! z50McX=D4PZd6AIeB$v=;mxAtypQdds-d%GKjO+Nh;rfA(zcu)nD)-nUs!AQ#aBw)8 zpC(P|Jqfv9FkE!b5&j-jF=IBLQ2-6TwR{b#y?ibFCNYdR1?#-!27Brb%a8H%cB@^T z!bWv?xaTg)NN8A|>|XgzX4qmRoN0V(o?Fg9N}K_q_WL6n+aqCO`j*0#6TY3ajnfl& ze_ByFE^}ME#UV=YbI`I!)??8^ja8Ag$Mx88D$AsaAG;f+SxaCUg7glHH66n|tpDck z*%>W;l40F8ZdlEDmkgt2Q%>9V=eaa9LCf^P78g@k8m{x3ySizs8mPl8{@0UcoY3>G zy`LRiKa6h8`(iyoq?0G9Q}&Ga&OgyF*iF0yDswot5l6`CkN~i-qgVpOH{S7yE^}Y#6?kg?Qzh2{ z@58V7I6~ zMQpa?<7OE=SPijz#o?EQ;u#uEJ5t^Tgd(%3Y2+_fGt7BdCJUS~!m@cXoIaQ~+3LJ> z`}x)D5Og$JZ)tSv(ottGyIT78&(dtu;=?%sjc@o3+dl2&f)^BD?YO-%_Atu z$DZ6!Yv%NS;)|LKWx4zHDbEj+{Dl!Ls$H8a>l-&dS5)&VUR&uvQH+feL9?XCPl|m* zEv0@e12L8zChu6_4ar2qYd9-Pbip%TMqBA#Ig1Do={2A<~wk`Z@Q3Mxl2fo zvkyMY4*4ZI`BbR@Nc^iikDq(DIIGLo1kkgOj^U8u`r(s7F=x>k*S{w9wtF)c32Y0* zzYCsYriRlQjxb$#yclv}^G%>;)yXk4uWS@7UJwjQ-UJEz1l_;?aO( zAcI#<3iR`^K*H{=$xy8js9G)IH1`lnb)O~O24@82u|OD*#(`rWz5b1o!h>ClFN6G& zdbqdbc{c@sxliRry)kzyepXy?9-Sd&kKWs}#`d}U*V8XcyRw&=onNwGfj82@(=@a@ z@>d9l1oQI?lO!KFh?|bZcg15|G~?W+juJct8do{W&m|v_e_+onf0f(@;EhdT#J_5J zkpAnIhmwhb^?#={_$ZD8#nsXHKo_N|0;qn$qN)7~7;1_cFqdH!iK2SVle3pdOJ`<# zQ9BXa?L)jOh-@hWFbz!lO!CEyzCU?;LDmMt!=kF!*G4A61*@W(prWH%!6oo0Np=p$ zRxhr>4f*@gW7W%+4OslV4Eq)D#+$yE(KX)5j%d6) z?@-9T=ALmTzgsq9N7quSX;*z9O7tgFpbM2P=(Zz$HFokBp&}E##(e>l-(5;w2bwPc zR=^}d_ipsC>N2)L825STfOh#}DD0fu6e#oTod2)5#>Jcd9&~YL=1lxHOOaD{8JRTo z7N&^GD3EfveLw#GSX^wb)4L$)lO(3t1X{~s{Vws~I$yH(Z7Dt zKdXoTIvfJz*Zol9{ZfDBd)bIBt}HnKQVS3`-246y90tNp|;{zg)ri2NuK+$C0MSQ?VJQ;rby) zJvbLGKNMs|T^iEHoh0wPJ}WlGHN&gTaAkTBTaj?$cWNcEZ-aT2k~{7dD(0PMSPw`+*TY z?ibysp93K=Dei-eo~k#yv*A#1{X(ID&Y822IGUy}SDH(`br_itHdJAa7-Ou<326_m zCyJJ)W6|FvXr^WI-W%uBsxPz-`R3|+8Q1Y48u8*U&s zZ@}ZXwkx&!_+nn$QL=+0w6#t1vVT$;b$YB934Iyu3VjXIH4m@zir8}jcnEy-|Ma^1 zQySU{A&$)wXtS8W3HQHWon!mgcKcsP1pe6F0Un)nF%dCwGIF%Aceb#z{Rj6?svj7e z=GkSV6s>%^D=;ibv9#6Jsqz)h@|Enls_Nvz!wV%bKf?VK1r^c2IKI45UCZIjEakUK z9CS^!9d4#Q#*|=wS{t!YV5%@cxD{5Y%{5dUj2Ano2cSa~O`i7{jwGi4n3T=nfp6IH z>a#$@j_2t^{`k_b@}bqPZKdhbtzODZ_OK3YM<)dt;X|gdjq_}?@f13pu2(zCOYp-` zyur>I{`}IW+iwjaVgobW!W+6~I+(Adr9D1Yp4e#DbdueA6MTy1wJ-XPP2qkD8EIe? zyCis(cMT32oaf6A?dMRE}Pt(6ayIrX=; ze<6XX#A8e^()4(Q#AAh-LVpH*&hO4Rb6w+O)zVzAdH5BGt%j zbFs3_s7e+nSB`h}h`1v57((AXF|q3d&Lm|-D=D&=OR(I+{Nl*^^)GPM!v~o1<5XF4 z{>VvAj#dFBzzFvBFWasErat{&D_nnbQa1ad!t`~GX*z#T3{XxTHBi^NGxT-l5HEPAMmC=JQ+ z!pL=aXC}TQhVoYGqpVK((|S_939e&y88dmW!IIqgB*&_Zw9*=AiDL86T;$~&s+p^* zV7VTg#2R=Z@I$kOKsIb1#BXNUT)X4`cFFH*qpxmVw`9dyGS#_;4Vk<3mFJ+VnD=p9 z=RlAVCUo}PFzRfvQ)syD5=<{kGk%HqiOb9OLT!B;pZ$Py=SCh@C!jn;@=g`ph*3WR zC0*4uwHeSU$(t;3Fbetvl~kC1#9Xb9y~9xbOB2UWq=IO(w7b8Ls}R<>xX^Suu0oa} zHsj*}J&yD_y!=V#m0{ospz1CCG^iJu$k3=-iOeNTN%Fz1A&>+>fMJz(CW-seUZar3 z`e3{+NwWW^Ihi9X5kCOUssAs{N$6h(7NFXxftiVoiLEn(l8LFciIKB~s|m2Z=4@eO zBC9TBVqi-qVfP1d@o!P4NW}`cu#4q8#lFN$o8nKzF>G?g?u-w!sU9Xph5|#O9~Pb? z<+d)>t~{58@+|6&F1wI=WVRc_++`D*7!hlp-gS8H<(J)cKCP$M4VpT*E2rH%3CFsw zq&Sir77xG%Z()(xjSi>>(Ge8w)d7FRf;iQ#4%5;5dVk`Z0K1~OTfe=86%vK-G`)bY zyx+98d;A=nqvqIk)ZzjTS%W1H#@sGsc%#c} zcs+@Z(Pdr!-Qp4Pee>2x3ymL(V`?<$A+ti)Bt({rj;jiuCjGmn^yL_G!p-Wo=J{x1 zNEGsG%Q|gmFZmB|S%Ypx=P)ptVc=CA8C{EEt3yUZhD;gjH+)(b!|yr|;VJ!$_VKgt z8>HV}1*+q=xJO?bkv-(lVMiB$Vloh^>U5Vv9B_H(~Ir#|0=S9{*m*kK&Trg>H{Z;!Tqt2JPr!uJ6-2p z;r!^)8I_ONPHZArJ=uI-9i$Ig6V<18EV2Z^DV8N=Ob7wd{pT~EpJdq;O5+9W{C&*= z*;S?qda=x=_4uQ95`F>oo}VT;ze~A|QTc%Jz!|+V3ZMr#VP$Q5fMVZ(;g5mA7)$=b z8H9^v7julQW&#lOL1l4B97xE;!6{52gCyOE^_4+OVhLwpSaQud=CR9Zf}lEs8_PCd zi>E2)F%Z_X%Hj*6GTltf;q`a_DWsy4L2q0F%(ppznQ#Biu=BsU1u)>67#RP}7yjl1 zs?kgc0Rl*(zo`I*1zF-qupGLAsIVwTpM&ePOzOY*2UkXfHe7uo7;7C6D@ogX|IK{s z$)EjcDK!WyGz?04@Jokg@^iBViw+a9I+=q|>gk$ZY_qJg)-YFIcbI$-Rh5?4cGj0p z)>qfZVR-2Vnmrwq~eACPK-V=z&_`F|K^fe@YRQ*~^cT~!* z0dZFlBq|O+Bs32MWwF!cklMZR;*iu=ERW4qDmI3T7kV~c$v#EU$-ppA!hl#S!B6e_DWE@6r1 zkcyn~eCTc(Iu%=Cl3axbYd@#AKY2iWOk-mK#^;v%>wwx#^WCD~z-Y!b+WimX0$^C* zg~eV_Z}DP6Ny#j%;HWhX0uY*HW+6N>=}0rUJwUy(_{>bdbEXLj_W+;9Hf7&hZsD*z^=iuMT)cP8%2Q~{O5VJXoL`hZh7os^zO*0J z$|o(XuzM)9auu;LQ?+lKK+<5MzGo9=%&=k%S~~L)Y*keTc-G9LO|jc)qDiXQ1Zu%w zq;75^hOEb!rVt+lb~RP`Y%`kUD#4gd?rc=&?sA}#Eq+;t;m}@tjwk3NjgyjY&DITZ zY+gNp)NX0DKE6!1F=!ocqt)d)c=De%OAU9BfJnj(IdYvJG>j9R)=eU9H&pjETBPqd z#L74H#LvL(IExpr78I80XEnvrmD5Kdq>DEv2qTuOagR&<=xFWZhfiS|C~FZW1_y`d z2H?cE7UT|GooL>Y0%kv$Cv@3|hx`@0-Ns#HR`MU6M&Bq|9c7>Z zS6{Wh6Ys+({6GTP6rn~b$&yQsgFlQx<|Lu~p3x}w(kn`4zBq+q59?&Cto!?Yhxr}) zM}zSPX0bf-I(h7jT4@Q?B>|};Cx{x(cj=ukg2V)ad=szsx9_M*Pw;28H6995Xpamk zCGRryd6`6>0cDr^;-u){JH9h&S4a)UsC#82+w9k-BVUN`|0I)0$2cid1FdrVFQ+U_ z|Isr432gi=sF*|xS^~AJME%Mf7OW#fkwj}9s-W7DKp=WoT5XEVJyWTRs;0KJ_dao{ z)QV#g7k>NQ>-YBZ(dP3xb(>9?LpbK@Kz(Z;#RXlvQvFT}D-w8o(pM98F~#(>hMUsa z>2kmlP6q3PmM&Xc(}807*WhG|In)__H7X ze@5K@<+6YKbF!4A<$)LeKEBZ;X%1k1McTq!0HtTT-q|G@G6$1gz^aj6(U)AISTo$| zo=X@C{A0+T3G%fEw3px@BhB}A^6cV=-;b`(R77U_SlFP@bXZ}e6-J7~@zEv_w=Z(g zx_sB2b@PXg% zDguaA9g8^a*lC1xZUN$Hqw$yamm#zq03J#DA5NelLZfPI6`N$l<4Ji6@m}8orFBu^ ziG?fP&Rx?lnIo4)ex_A>dr`F-*Rr#}Lb;&FN&rMw-L8t_E-S7xd*+V%h7WHPW4XZs#0G@yB=TO?6aG`k=8NAkao#x*tAr_jNk1VqUTb znOb+lh(=bL-%-)5aVuubjS3hbFgEa3&_vgQBJJHzvCqOYi6ax(phznWAW`?yY)~ke zL{ocM=kWclPZ4uF^^f9Wn@)DC&XAeR7++JcPffBFyW74!dPE|bnon@LERRYf;RF=~ zQ851r#-m`ia?*kEI_NLKIQM_{ihqFd|BXMgR29`w#nC?4p{*sR!xP~bw-EYx9w@_JX6gsE64uBBf!cIT0T(G8E+fix?u%PR_@(KK2@tC=C;q7 zl}w7~pewCGwj@q#Z5`RgFWT#ne)tBH^{!}(-OLh)1l oKveFPto?=7N*A1c2%g| z>9kmfg-?1a27~mYc`(6Ds(N|kyzz+JjH~MkZ+_}LJ$ePHqwm`3bSG{Sw9-e>CiF=l zWW;_4e!C?F#;dBliB_lh(&WNorW65ENJRr3#A$@CWd7Vyg%Z#)QH}-Xz1K zrj|p`u3WCnskc(%Zw^|zI&@m`v%OQ=vZKOqaE60D;BZ(}0oA*9x_SPWigR_Axshz} zost=6!-zN`o5STH3!$9kmxQJ&{R84xWd?6xo!hK2)S7Fk*?o|)m>hT--qc&N zX*9Jeu~eh8?RYd0~`Kzo}FlBHLFTBHCyDs_aNGr1<7Bu*k2-VLQV-iO<0_ zWqB%RP68wY-`m($EFK6@E(d<*|_+iWGWWyZ^vzl?1abwoqzo6DSe-~+EX z2DkD#iCxS_#bS41PD)xiVROwRI97R=jZBsNs;YjotEcDpABiO9oAToxFphvh{T256 zH@unu0p0(16iEW=$N`5Lne;g%Ao<><`~pNFI~qHJB#;4)pTa%0^g$y`))JSV3Vyhg z%!*uoQ0-ck0q|sa&$>JkI2sx}E(xb=K1Dmk3V=Oh!9bhq(ht>Q_J0*_{F3A>hb>g( zm_xkIC>}t(ku0U%-r!&B<*kpDV3Q$id)G$a`IV@DN@NP#(q^9ER5>NmxeU*M*7k^{ z`bm8v$nlP%8?`f>{^@AvMaJHxFBTitOp=YRh0-irqfFJ7$=q?E2MTTfZCPZwiEiF? zU}V~JJoYHPnh#C(Sah_CUn%RxD_aE4Aw(GQ6hh7DU23x1n>)F-ja1gQ2B)tPOB2+FS|oHwsYOttWUN$-2DX+G1P8@g+=m$;+nzA~ zxm-&HTS)?Phg;Um^%~Qh12$-L>5f`2hcem}q%)NH=R?k*FS5~W>u3tb6{5ZL)i1*qjoX0xB{aRKFka6sbNT@LWq zf=C97ZYtDWgDaY^Y%E~2(^@Lpgi8krUQNCotMd_WI#cMnE2=oRwH5r;xcr=7irlm& zSZdG9o5WX+hPI<5`*rN82@j zEHPxa7*K0sC#v1^keHn3=q z6GzvhZW$pNMjoIObs19w;Yl8*((tmHWOEJ<=xLyyv78B8pR7){m?&!unTryV@0L6a zb$(HBZ*4y$X1XqNw_EK1Az0ZuF|-IDXcI#UuGYRboKUw*8*?ZM-o@HCHSES~NPOI* zFPVk7P*)$)@fUujj4kP`fd)l~ePp!F1EBlQ+R5e!T#HM=W1vs&!PNr03Ck&cA_H(!aaYswWEr-{FV_zTkOAUWABri6Y^D=}*QP$<$T#X@5GNBGe7oS_zQ z6s{O;15?|dSR80M+E~bZ2+a<_Zo!+`#3nc^)r|Q}~-U2F8^*4;?K2S0y zklm7>zvlM>lc(d3>|BQY52tYEWrwed^wQDPo&i?QS7S`#@um9u_;%H#7pZxHEX&y?9MJ~s?|8KlTVX3 z6gxnb-BdQ5gKQjy&m9unMC4PWS9xo}?KoW3z&Xa@;sA1#YFGg)>j=W;la{a{Y3;Uv zNx>TOGYUKGM9BsEC|sWPrH!A`A(^0^)47q19CQWr)-YRX#kI?Do0qVwI?tiGL%~|R z=|(F#^S+$%R4zfgCESYzL=^>5+{hBKXnwPBhCI1jVVl^Cop?FWLkRiK4edD z*$qv1od0ZJ$qRKFJX(Yf&V_oc0fD1-i`@%GvM-TL;}oUQ5M$5e7yOl)`N?(;2b)5V zM6iOH`@2>uX@=w4CA!-2Ylqk9oBY5O(mHAE4CaU#)MXZtBP)d3Loc~QN-k2&4#5Fm z>+?1O^&5;U>%v(I%HTWfqQzxle5@QY-{1mA?QR`FZQxzM^cS(|D8)!ZT))lEWGp?= z{hy2lUK>rSKkDkGzpShOUphX2@BI9eWvZ;n0j17$N(4$41i#Rb(C7l ze&JnE%mzN|-jP=HO60?uv1Ll{@P09GE!~a}8ONzMZ%yxMBC9qM?9UG&_m6ARrgj4k zABq}33S-A>w*0P+B)d{+Q@|l2)(&9j?PXdF_wXCOX(1iauc0r)ophmbu9xdg&>^Gf zHR|x>o>t$T3Hx9CXpyw{n{_oQ)7^;uXm8tAU{<-KoA)q(aedlCbA{A$mCdcu*YrQU zT))qhJ%&B1sHXKn7g6fezIy}I>6qd5yMi8f6dS3?E?16}JA)e8q}aTqt?tH1KByYo zOQ&DFN7-0_g8-8U(|%9O##o=V$U@Jo0^_KPNZko%y&*T(czo09n6&g)MC)mC`*P5c znuY$t{Mu)zi4=K!F{W=DnAWmT>Kr~4IDH+;8PgV~$Wt9`P(17eP8 zG0FwQG16~C&VdpxjA}OW{zR<8fJVyO&pS~8c>CCDz%7RDGLBE2A*hAWnFH$cPaTdB z6WyX`v~#>HIytlV1RDX3hkQFV5jBiG@`owEmUYl-j%nn~3TDvo3&vxMXDW_Z;{m2v zU2zj0#h;{hIOQr(c}pzVLLBDMC>SMi99ei-Qv2}cXs*dkKJQSZUbXJX#bu~y1J)yV zC`Ko#ZKaa9W9IMBlDQk!N^mUHPjXv13jIX3I%|_Bcm_79L<2?%j5+$8#j!E0{UUAT zYZoDZdS*pK98t+wrQ+xWOVh3erjnz`bk3fJ07YsBGr6Zrnd2AkcOl+mKEVFK?$hEE zg6^B>d8zZuH?y%nVLH|l#bM)FdUG_r^QokA^r3Oxvkl4Nb%&q8yM zpwuVULhI`&4pLE0DQ@DM z8&pyd#r_FUJ&u1+ojm?S(*~?4{rkx%uE9J!WB1V3moZ@!LD1$@`3ck!bf3E?XXvs;=^2N;>QIIEBwjWrEjlw6p*0c1xS57lStk@HtURt@aFPzR zxh!!j3Fj~*h_r{0QC#Rwp1r70-FRLS?d~NBs$QxU^`oL0bloJJ2(a^k9=ksrh1D2= zz6Y4@`u=iENB*xJ1qikNOV9T2nrXpG20+a;Bp=&`!bu@D_NC88kx?Kvp->4=L#)uU zM5#&;bV!i^&2_yyADU-RD6~g%AnV>J)AC%VnH(+QXyj)&T|MeNM3iik0A4M@aw9)& z1sc6_ZSz4?KgPLZT-4MiH%Uo03^IqN`AfNAGVQ^Sif#2EvBLnBIQQ|M7r1*;Y}#zh zbfwiGLO3XWlN>s83qm1Khh)&A$1cHyyX9PvT2!kvZDh%%!H$6ddhh;Dc ztO+!}#5amKT&;(Bfz^E_mF0Bi7qV)uMLWI8_T)V#Ccje!t$)US9 zt_6MG+(kLpwdgSaRd2vzCqUJIm5Bx4uXz~+WuBEkR!g8UOI-=`b#m5+?pSj^7CWo1 zT0iMRi`+Ac&hwZh+&cX&@~t25a1@OJ+&>QEn73~5 zFI+r`?mg*>`(U~B)h(X=tsY4&jiu260uIw(Tal(}1Ot1OeBJRp2>M-{77v%#A6gyH z6Lj@B4GP%lnF?##FtN=iOikuSCl?zsj7x;qR+^pB5evR2Hkq2)wGa9xp;%FPa171T z7U4rEwOtcbNi$*DKSqr^fcLjN&N6*6qF8{lHLbk0 ze!p!et{pt5#%+~gaJ=Ey%lNv4B;1B$R3dFhAuBKDC+SHa1i3+1WetclZ2t<@Q6LGz zg~K8?K_qWvkSK5sI!FhgQcV8xwV?D7a1Xino8;OkY|rO?Nx(UUL{uMxdP4EOufkST z>5ytlo(dL?9|zH-zbulAKguFkBs1W^8=r&?ewAp3BJIskAW3l)tI#Z`R-zQs*aDg`Y9+}_~yy&Gr?=6`7m|F^U%*1x4#pfa`;C=(LjvCXn()pGgbP9UH;pn?RT zQuqrzXf*|6Yu?6OtyyE#_P*s#@~Ri{3UVL^nX-<^yiP~L^hww;%?*V_A?d+%B6 zy>ow<|G@nE=%bC+TW@X0mBLOK?Km7g>ieuJR$q9=v^Ko6ze=R5^pdYvh&-&vFmiI% zIsS`bZT`mJmvG>cm{!A8qZ6v)EAFKz$Qs;Zj`!cUvsl8sLcM=mP{~u01CJ8>yeBF$^B@WcVD3E` zIbZJ2PG29N_9%aNhJ5{Y_}GG;1CT+KUUleNp?0dpg449Onj1)q=7jPiZ%8tUDXvCW z0sTI-B7~%3OfcrGfkxPfj7}XDMCm>_S;9g`1?|9y@I6RHMTwoB>NoNoa-l2Q@zg#| zzyq5vF}ew;#E5C%Ru&U&Tt@_hBT<@BTc|uu77=$_bwAV!>TP1v^dS+VNGJL3VpN}^ zdrG;hL^MDd0An3hfGsoXoHr(UdNn)PJAAXf>8_Ka#3O%x`^1~rtS!rC0>>bfd}~S; znl7ZdKo*6kXP<0{jXyxJjD;y#U-S4NJ)Ye9;YN3Cq9Tw#8Ra&@1Ox;v z0qrAO z0CW~Q?^)*5`1VLx)Vags_ka!!ps?w?B7G!HDZ8G5!ZlO!`QBk$bsB;sfU$~mbX%ir zuxE%oQ|OpIFF4#VB%Lf19dZYTt9Rtz<+C?&v+5`(<)}rm_V_P{d=c@kn^858u&C&+yP`QaQw1xCvuts{^G}5 zT5expiHln8kGxC(6R++Q3vWPpq8Nlxv2bzu=2pMcCFu3*6sCUy>tee2sE?!*MJ%++ z^((io)^?>hM!@UqTah1&sOl&!*Uu?{zj!-tqcV(u;}vLDF8RQmf~kk2NE|Di$}*#7 z(S0PRNCQ^~*g6NmX$9&m!awW72$JFpR*r!F3s=Rq9i5kaqNbbA-{0SiivK;X`tSb* zL1#xNTkHQnwl74Byh`dSu2y&mi%`U$7grmF%5kqb+H$DmYut zyZZqFMFHu77A^$Nj^S{(aktWLX#)(9#u=eZQA#Oc%|9noVcICBJ$$ja4Q368oMpjW zHTU$-1pU{fLH%oH<7*d3g1uuq!=A%zk2`kR6oTa`q`7`JjM)X6ur8=Gtg2XuwnsT6 zc1SfOd6ycI1o@;%GQ*Mo_QV+=pw@s8=^*1)C-+%zt#V8a)sD?13fRzIsk~RpXJTbM zv>UW6v+D}+foE$G{<38TGd6rNNq*b7Zkw7f*un^#u!%$D?m?9$p8s;~^t^6lgofTHgy;|*(4()`?v>zgsV)%AXMW->5+TQ)aRb zp9<6ey-R2Q^|~#hwrZ=gwhmX2`$p^R>30`Kf}txWEfz+U7<0*HO=_HeW|L`ib-mkl zg^WlZO7i~YlQkW!3j(Fcqcz>ncp87|WOQ`#eSf|O{sqi6XAjPbI9XdV8W*M6Qn>?j zZb4?&QrsgX9}I{C)Bzv@NcvX%4qu=gnPr-q~;!^rxd>*1B?dac!LFfcc6_F+v}7Se=7Q`a(G2!iWx5b@EZY*6Ejz% z*V9+&@eU~3#_u`)AUCQZS<{xwwN);>HPXK2I-=y}SLz`Ij=mBi*Dxmg=$T_w8W*$; zsWSD&3Ko@SMy&TuqXGzNKIWTJX<3b;6}eR(cqD%$%7_w5KId_Q$lP0v!*<6?GD!JCTTyNB-^`dwDTNC_!4flw5s5##Pz z3~nK_GRHT{L|8a-Wf-_OgfI2{8WC1;RuJG4QVJGP1mU9QDZiER7RfPCPLbpT8LISI z3Z8_rx=|%`;n4|H@|iVo42~HcFgJ3~$b3WG_TWO_u6d06LMg+KE`W&1Z)82>5T zKl6$e{=6jqP05@nd|EKSC+SQB1`9!U3Dn@}tYbq6!tltDeVy$945Lo3iAfEC;_?Q<{CZ?hnTXoyPtQw%xaB|aLd9a?D zX|Y`aU1-wSNFuMepKjm^U>wzgvOH{Bx}{!Y#MZRzgD2`VP#C2UR}d~P)8yn}?PGTm>ENC*9&FxYwU#hGNkUOM1uRX3MJ#k< z#5?z_KNi{dr`>g-@jD=;g^eGud&BLp&iBAIu4@f&M(6Q%&(AGN*WnR1Ps; zL$Q_Wf1rknDi600B6vSMF;3^D?#LjdH04g;vDCmr2UXU`tNbl9b&+|Mf|c@?2Y8ms zl%6IFZ+(uScsH73xJW5v2kilX+JEkAPL+f~) zGB+k)xo6`IcZVAEZjG_PT>JqbZQuk5OtT&MW1@q*K=Y2-Q_g{AvGZc@V70IOSKk!a zBYVN)_7bpO0Ot-uUfli)6r!3}3lh4MjuGpIx7gpX@><+V?7-;j6q&G!r-ISxWyQmv zHI4mSZ1v;p%*|S)V+ndZN;2RjW;t&DWdX$X>=xMYc_8Bdk%syI0!@Epd;RZi{6GF2 zs1hR*xs8w@v^gbbP4HTg$P@z*IQS+LbPZK;Bd~bQf~SH!J}wt1qX zT9WR0p!Y{L*-gwo07QxK8dqc)6pS!!wJZwIKBZf_IL;Xv(I@Y;dtJx+suVDR>`vl< zF%d`5K+Xym4-J(MLHSb5;#S%$@sSn}_n*N@1;NNnTnRP1zk5Du87sO|tyH#c2@>;O zYGu;6$)?65e7MZYO` zE4Mj>n@Frnet*v}-AcKk(gU^J^+bh-?s_@P3@=^@0vZX_`z{5Dp7(4XDd z{VpwpApM8s_)g|p5K76i&rjEVo^ke`4(&fitB8Cc(F}*(1x$(<3@P)6O!?iDF#_so zVPWdXjuhu~Kk#2hn|`)grDhF$O2g3s$-on?Fblw2H_GQ1oNcmS_0JrVbKwe;s7}Z? z7=cH8x7@N8;gc>w9%M!uI-t*=@JR%#JoGvlp+`Q~J<-xk2P?u`t)Kly(4Qv`P;Z4P zhUTPR5(mNv7jg0 z^Zx0_Mhaid$zR}Zej>9u{}>kIo1o?BAu??|741jm5j$c4_fUXtpk|$%R~eL$^HA^h zBN(dBB^JHtUAc7_7W+zm@ovVU!+H!EPzZy&cs!6oHoUfYtcPQ*ibAoqzQ^rIyWWaC z5XtIO)_4Ezj7`srSwzL8ixr zav5ibbX~@i$|X|AUlFv0Ur;KA03%{t9B3i6m?J+iDHXGO(!%d5&M3>O9{=Jun=~F3 zJKrVn1bXMFMg=~j|JzIq1!$k3Zk-0njuDh$a70VuKHK)(!a`VS9k;%L_7gmDJQg$qB|`@7~L5V z$jj8B5WsKAdCs>1Z8ftX{YO!%wT^6ntln^U7^7!3BMX-YYA}-9X6n~+u~*dfz}d2U zkH?s&vur@!x2~?Sw4$8tuE0wDo%mHbZ&B)mEs;D2GkoRZhb3>?!)e zGr58e$vvu`47?e3IDU{;o1Y|9x3e8yd2cvb$pU0vu^sg=eKrwYOIwB;zqyp%VgKGJc(1K`Sy5x<0qi2$i*bQx=8@yG&z9Qlc^x?i1uv zpbw^mlBMPH1w5lg**F~@?R=^CC9UCh%7mSwYaA+M>>bXt-xJ55rO02{$x_v@kgLv? zxCHJLY?GRT5JpfJF82v2z{bB1A^CVLvy)S~RK>dnys|BQCR^(`@<1XeVuiZPaQbuOThv_%gvs}d zSAIK>p+fP1Y{3j84;u6wj30|9ooUM;^h~F!8{++=%j;^U;Q!%)-1m=2g7m*H>i-%M z`ft`HNJ;z8okNxRN+&BpJ|TGcz)0y~IDSR}fqtf5dRmhfX5-n)0!!%ybYH;7uU<^s zmMO>x9klAVPh;B#NPc~M>4}%DhaGp`l?`8?*KczDiqPa#=3;v4f;d4)IC{oN7-19= z7;$bT2FgLX2GV%7)@?2|gm4q6FdzwKu2$ei$4!>Z&ufGuzQ#E02@wk&Nk>>|ozxe= zxd=|DSzo^0dM`A-mKolApb$ekPuOCTdJAASVXAdna`gKlJGJi=9o%%PRe_aISHH~P zf;S{qXQy09_0X?1ZE4zKl_Nu)QuZ(hdHbm~Q_(T5ifB~a)R>bLbd0TB_u0`#%b4l@ zqypdCs2*CTJt_c%loqYIjOEZ;E@A?fV66s90EKCO2ez=XF2OAtGbb<9C#lq!8EuR8 z%0~lBv#UuvcUH~VQ(LoAJq=%uWhc((RWc_7wWXjMIZZRDxc`X3{vn zxMItNn$lXggj`|2rMM~=xgnB4%h{)ymcc)~*EcxK8bWJifVPtg7G!SZb{q>-*8=o& z|8(Kk6Yy^lG1YL>kWBne#jwL72F_A1A;1A?N$rGO!c4VXSm6a)=*}(Iu+m|szJvPi z<7zJ(N(gPfjnH5~NnCoKPT(rjPqdCCJ4nV+{9yG}l42v*y@u~g9XTQ>={n@^0(rd; zP{&7!#OvWu69I+bZmv1mgs1@mm`Cii9Z0Ib~H2$pI zz<|anmF>u|e)Ni6DWbZbM2Kw!%sZ!Q^JFEexw0gK|G4| zwglS1o@fGXgr>wyo}4v*^P(jn(Eqh!`L#({h#1QACGQ6|Am-RCY8_VBkCloU&e@tB*MqQFJ)Qckh3N~QK3XnqARGa2gRs!Y9lHRcL ztiV{#1rQ@zu8^UVU(Y3SDuR}rP|wz7yM3bt$Sel;a7;31l6r&a>zMgLD9;;JYL05-&86xG@VfZ#)@pG4(z-`Sgqb^$!HS)p^8v}`g`f4s!Sub|j)tD8 zey|7Pm1%4U3Rv>0kE-XeFbry2Vd_?q-FLXbK=~m#4Z<#!kEl_rG5k60_sIMczfl6u{OoqZVg+@^|H))hTPAc04`Ve#jJtdwqG(BBKO- z1q=dt^zW}v#W|x9ton{tBK>UVOypdYhI!^&o(u%ikQ>SHd)=xS;oWvOae32&6luom zE8MY%rEw|Zkfx5@X z75rfF3T$KAE;e%wXZsKj2E)qVqxSFm5&gk7rSfpfJS00~ITD{O_{)3Q2RV1K+RQr~ zP8E!aY!o4pn|`&&`ypdF<1pa`)GC*k9U&d7jNi@Kul0xfIVX?3F^WI``ccDqKS7dL zGz&NtpBI5juv$s3^)=cw{c;ZjyoT%L9WuedA(zO4y6uO<*qu9ECwu2J)QFxo*Cz6Y zToM$B?5oMa3-*nW%#Uvs3SZmRaPYR$;9~*>^a$o_HUmCRkAa>zS|PXWB%xKKCly6) z@mEnRwwB?aPteE$zY8*J@pj8$_XJ%=&STi%=nFkU=L(GAd&~-4!o6AGJB8pkPI`pE z-+}q5k|Lkw+W@WS0Q>5}U674F?51+o7~vb)Q4V!s95xe7F9Z&F zdas||OlV*9{fYIYq)Qq=KO4*5|BaZR-_g<8`d?-rD(1FUe}o4KyBQk)7x#}s)j#8d zNK#`U2^aCmbV?WKz=br+-SP>9p6NffFUf;j}5PT%}w@v{!q@=kTpz_t)kEZ1J zLdw0P@b$v_&8%e$0Q`hgz%O`83X0jXWbt=D#6OMF!-B*=-8&)0?pVC`Mn}4YWQMx&O9g5;&D&QoNrk{-B*SxjT4F-G6|hHcS#OSCqF(uR&Ko zy0ZbgRTsdLOSdBfd6y<^%p#+qg?pE&R}bl9hx9i!%Lf zo}&!&)Qg;VU{u_ihSZC!fXoO3Bx+qd*J|PfX(C{|GYG+6>=-a6B{)J{j-Ngc$v$Rs zR4++R@~xLGZ%&AbsaoBp$1w!Z_uw5a!0N+zIC0PX zjg>~TlK>iBgny+y0L4_kqdZJ9s@Jd@aeZBNj>eH=Z)@S?OV<0vy)ncYj;h_y<;UY8 z_j#5on-{-yLTmBP^_R11#^8D&43kB8@=g^BzJ4hNNosGhKyM13FLk9w^21vRj|X%2DKOct-Tt*$+<3!old9|z zLe-=|a825;dwC=Xp+)uzsz&w-`2Koj4F`+N)$PjfRlWJ0OXZRfrIcC679ginZnf55 z$-b(Rd;^{|!secx?+{mWfc#Y(=DT@h#KP)n^c4r#jUQ)8A{B%?!M^tWNy5vxhecj@ zumvPbfqIlucdl}x&cz`3gvwH#&d-}tO)3UL?xCbp<@%D<1!&K{S{KsB1u{#s_5G9u zumMJdsE*MdP*xjl-pGVphXr|4Y%&6 z&wj})Uu^3t910}B;w+0k^}-dsgi`e**=%VJ;EgGUB%@I5cX)pFr1LoFw zH;4@m(K4+#+QdvgvTBLDZrkJn%H8llpZp^(a&IeEA4F{p9)2h`=~g*DJPMJQGm%Us z{?tX>AOg60Ovp$m4lDw&&~^k~E=L$M(eRZ0;Uk69tL5S2$zgjNVBCEA5ZMgq0EhWJ znJ77`8ZiD{$W&?a$PP$##{P#2FjUJIMVm>@kuMf`T9hzS;cQt1Ddx))vw>ZWU*|Ng z+7k^J)xcf258=kA;{!;{_m+X~(7U^oa?(EC7k}9KBirEVEy7@DX`yoWLaBjOP*|R!r`$j=WmA!sV-$495rkVP z#Z;WdjqsX~6LHRKBY16lIQLk zcz8gt?58O6(s4Q6alSt~p$O14M|8N;%r|l4IH_qLiB;}MSm99d+ZB5p)ZNRjKVDbf zD9ccLpD(Mte+(M^o52OYgM+^N|7&n7qHk{H>|p#CQdPEA#8gJ|u@PH@Ybt;ib2m2+ zt0cmdD4SoH#H+Tp_<{`4)Ho;UfJg(-UR|4fhJA&6_xG7gjSI`Pe=T5q|HAvMIO}LT z4~I7j#%Ox6^s;*JxkBW~>ihb3L;GdDcLuG$#!QbK0Egmfhwhs2vs?;aWIu}mcVIHw zfHtW%@3I_}C-ir39z(XsRqC?(Ngu~KWxc+hBK9L!*msC4x-e0BuiOw7L$?Uh=vF>qz`ry#4u$bVsYD{URL7DCXle3)7DBAXDbREw9*&q>Z?Y&GF@FB4nqzWCAH?o zASEU~=e_-6rRvf`%f?p`vov%m?OHl9ZQUx|Ibh_ODh(8`Ik(}ZPMXjsD^)j2#Dcx_ zUmI^R*+4DH(NNkO#^B->z^3ODzZ#EuQ|k_{7@+YuyH(Lx8s~nI-!SjYi?NhhqYfo5 zokcs-x=t`)pa;)8;|(kMZGk4G)#HpvULdVDm_n9owawjbE0AN5V_|?f-TVD9ri_=6 z8jhEk#Ek_3M zV9f)wO1nPO~!>aH!r6Z00=9qoI1c9Zl*!nhvZ8Z|-(Ly?Hb7IVs~ZQp8A z7^HhrZl_k?xrxPFSROoLeMc{&`~4u+bF{9-q4PA<@XSdTTz-3pnQyC(M%dy{Wacvn zsgDKD9KYL@`v!h(&Sjn{hbS=tb|$?Ixujh0!41(K4d<7=(4Yb)HaAsCeod|6(^>DWqW!a+6l8bv=FP_3glc@0>c^t(-umC{?u>b5LFOKed1l> zf5f}2|NUe7zuMhN>pK~m89V%+hIh(;t*%fvWWwZtHU8j3PHe(MzWUl=uze;>LsE(6 zmM#;07IQRe#KF9o{fL@kCMV9+`M4>H{*Ve|461>EyT5wBZh2OJe}0tRwc-0ktrrm- ze3xicpC^j#T&NaKP)FdKE22>+YM%nq12F@4$mdYqPSAdqqdEJg$igxyKNS?a>Vo_t zvAqhd(QBxdIbTKZkk$gQUqK&XJTKjrR80vzw=Yfl85u-o)QnCrF-l>Irm+Ckxn!Z# zqOC%~ff`d(cDNE&8QS5BW`C>^uRqkcVhQi1!-!>r8J0zB?=aU0+wHic6_p@6JCcFF z#ErYKNy=R#6O#OqU8_ceIuGWUN4XTrtTSx^*lw^QNIkAnTu~0$Tl;~8Hui|LQFdxS zM{eYbo`OyVFg1uNvoS%|A&DMh6!OXZCYyImd+ugyDj{U<66>d*t2*W>-la}eoh9pa zCLm6ubg%6COK()Ma0uA`3~USvN%c^bL9=OC*CG$ogr@4%|5prsh^Rc($Y<gn_e zeeqE{eH!%$gLSZKGnEY`!FgZjF!z}1@a&;EEJr0XRVHi8R*Q}K%p~k|nL%ikc~K^Z z?8`B+8>BpvF&F6Dy-4d?u>MuoiuxuDeSyc&0Ni-UdMX5418%@CP5AsMds*^u#yM*H zH|>>Xf~**;W>w|Wr$wZ1U8);eeG-hhMNIy>qhT40ghF7U>32o~&z9{t8<6io<7Z`liNmNe^58hryK zyaX+Z-tNIf!*Ad-7xpkxnDqG~lQUc}G|9(Txm;cmY4>fCyQFvCTcDu3j=%paPN9S4 zbouU+rh5Cwslne#ZvK^=@P{zc$;{SB!B*eM+?a^|-&AX&qV*r@c^{h*3rI1S~5EOtyrC+j&R1PTE2JPW^Aq#*I{*un8_V*Ias$W07uC6&A)W4lP ze0u*pAG)jkw#H>%z1q|t8PHyJwBHaO=>rxR!y0a%bG?fBB5)}|o*QhQaX>)oG?1Dl zd?O%^C~h36Vvlju{k%J6sSi_s)JByhF?GyqSfxD_1uRhQ9o2IAV-u>r6t5zx5WzO= z0WtSsM|@b!F@xCN1J2U?VIarQlH`;stAJ`kzbK(*3TEKu(GaV!Hci-;)G-#V={a9E z^ij{K$N_cbo2~4)k`=@lxTq;GMFAZ{WSkUn(7ZsZ*&w=AWSc24D9eD3%Artu!ab5@ zvE-?`ugxzpV)GNLaWxayBx}#1IYrOWhHxFMW-CvARe{Vc@7Q#RK9sH(j4m_OS+hl$Yge^x^#NiL zRdvs9&+y00b5*$m&ZN!+9_S<%1T?)!W4U9VG_~M2Ld=#?oun=Diwf#PEWfClS7;b- zo~=vC`-ApQwmw9x8?CJ)I)}u)MuvS3n}ieTRSIhy7W>#n5<{G_S_SGqsR(|;o9yjR z&DO^Ml|b@;DWFd;Qgb7HC)>a50V7q8bde6QKVYavYoz#8@bn^St~L=ure?|J{lNg= zz>)p=<5Toqf7FgZWoGu}i>5Ukc@uRsT2B*}N>30);!UUBddz$jUc69b`}j_bo_G8A zquy{$Ts*vO1hu6}p|^viz$rZPZ1u6ffPSAduJ<;m;>~ zWFP->&?@jtMPS~UD@W{V^O6SdjhlLmrUVqXw!v8Dh}^KYg6`F z>flUl>2--I}>3@D@Q zOy(PFyV-RC?6lnG#u*LBZamO!zVzQt^;t31Y=&XP$x7O#ttJm{>~=f~lTok39@dDR zI=SO(75NL)`l|7(?YU~ck}6L&Ge$?#31<0rXVp5SZZ_0q*)%uWN*TO>?q({gjtwsR zH0&tBU zsDqn;2x67IB3BOpZ`hlXQCI=e+9u=qT}9C( z{sEP5W`Ko=g(!S$@7W(eC8sDPp5)wavyJ*#7^R!~$h#97*DG%IYd zl1}0ia6%QQAU!2jN=q-5D;=Rf*t=5)HGHkvUYPqrQ^_M$(KD$dzlD)NPPXi8y8ZAN z01(`kU?8rGUt*_C7AHNZ5a#?DlRnN?x{JK>&`RaY!fM+_MA;ND_?REcyEVbF&T<8_ z&TvKZ-VNHSxRmAwYkSzDzpYMXj{sOsIS`{}9Hv)Vh8u8{A)%J84pxXGNwS~mg|I1| zN+|fT+mJb07RFK=rb+T1)^*fkz3p8n1`_4u?Eby@<~;S0kL<%{8`n)laAf(5n)t?k z{3Z#oOiKZ4yw)w&^QG)-c*jo~rK)7D8n5uu#y!8ImWQ7s43y6=ySE^ql0E01?x<=i z^)7IYewj2<8QPen<0SQUCfoo<(i5#(Wv*b!A{C~j0`rkdr`BDThM&!uhC;4&0XLAp zjfsOU=bIEd2=W0c)tF_}PqiIY2JnN`zY=rxW{)ut8J!FCz=Paf?||*Alb)mT3^jhz%47%7 zlasYjm4?X1iX&uQerloz&`Rz3YA-)Y;C5OwEfFoQz8pUdxMrmE$DjkV0hPp)ZE2DM zGb&tR{0uCsHl(B&NzQd-&FhdjyU6t)Anq0AP?f9P!oUa(uk6l{SZ$6-L!M3AicnLm zs#xMD`3ZZ_LweH5L@~nlebgJ-